View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   * http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.security.access;
20  
21  import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertFalse;
24  import static org.junit.Assert.assertTrue;
25  import static org.junit.Assert.fail;
26  
27  import java.io.IOException;
28  import java.util.ArrayList;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.NavigableMap;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.fs.FileStatus;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.Path;
39  import org.apache.hadoop.fs.permission.FsPermission;
40  import org.apache.hadoop.hbase.Coprocessor;
41  import org.apache.hadoop.hbase.CoprocessorEnvironment;
42  import org.apache.hadoop.hbase.HBaseIOException;
43  import org.apache.hadoop.hbase.HBaseTestingUtility;
44  import org.apache.hadoop.hbase.HColumnDescriptor;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.HRegionInfo;
47  import org.apache.hadoop.hbase.HTableDescriptor;
48  import org.apache.hadoop.hbase.KeyValue;
49  import org.apache.hadoop.hbase.MiniHBaseCluster;
50  import org.apache.hadoop.hbase.NamespaceDescriptor;
51  import org.apache.hadoop.hbase.ServerName;
52  import org.apache.hadoop.hbase.TableName;
53  import org.apache.hadoop.hbase.TableNotFoundException;
54  import org.apache.hadoop.hbase.Tag;
55  import org.apache.hadoop.hbase.client.Append;
56  import org.apache.hadoop.hbase.client.Delete;
57  import org.apache.hadoop.hbase.client.Get;
58  import org.apache.hadoop.hbase.client.HBaseAdmin;
59  import org.apache.hadoop.hbase.client.HTable;
60  import org.apache.hadoop.hbase.client.Increment;
61  import org.apache.hadoop.hbase.client.Put;
62  import org.apache.hadoop.hbase.client.Result;
63  import org.apache.hadoop.hbase.client.ResultScanner;
64  import org.apache.hadoop.hbase.client.Scan;
65  import org.apache.hadoop.hbase.client.security.SecurityCapability;
66  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
67  import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
68  import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
69  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
70  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
71  import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
72  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.CountRequest;
73  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.CountResponse;
74  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.HelloRequest;
75  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.HelloResponse;
76  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.IncrementCountRequest;
77  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.IncrementCountResponse;
78  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.NoopRequest;
79  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.NoopResponse;
80  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.PingRequest;
81  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.PingResponse;
82  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.PingService;
83  import org.apache.hadoop.hbase.exceptions.HBaseException;
84  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
85  import org.apache.hadoop.hbase.io.hfile.HFile;
86  import org.apache.hadoop.hbase.io.hfile.HFileContext;
87  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
88  import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
89  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
90  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
91  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
92  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
93  import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest;
94  import org.apache.hadoop.hbase.regionserver.HRegion;
95  import org.apache.hadoop.hbase.regionserver.HRegionServer;
96  import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
97  import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
98  import org.apache.hadoop.hbase.regionserver.ScanType;
99  import org.apache.hadoop.hbase.security.Superusers;
100 import org.apache.hadoop.hbase.security.User;
101 import org.apache.hadoop.hbase.security.access.Permission.Action;
102 import org.apache.hadoop.hbase.testclassification.LargeTests;
103 import org.apache.hadoop.hbase.util.Bytes;
104 import org.apache.hadoop.hbase.util.JVMClusterUtil;
105 import org.apache.log4j.Level;
106 import org.apache.log4j.Logger;
107 import org.junit.AfterClass;
108 import org.junit.BeforeClass;
109 import org.junit.Test;
110 import org.junit.experimental.categories.Category;
111 
112 import com.google.protobuf.BlockingRpcChannel;
113 import com.google.protobuf.RpcCallback;
114 import com.google.protobuf.RpcController;
115 import com.google.protobuf.Service;
116 import com.google.protobuf.ServiceException;
117 
118 /**
119  * Performs authorization checks for common operations, according to different
120  * levels of authorized users.
121  */
122 @Category(LargeTests.class)
123 public class TestAccessController extends SecureTestUtil {
124   private static final Log LOG = LogFactory.getLog(TestAccessController.class);
125 
126   static {
127     Logger.getLogger(AccessController.class).setLevel(Level.TRACE);
128     Logger.getLogger(AccessControlFilter.class).setLevel(Level.TRACE);
129     Logger.getLogger(TableAuthManager.class).setLevel(Level.TRACE);
130   }
131 
132   private static TableName TEST_TABLE = TableName.valueOf("testtable1");
133   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
134   private static Configuration conf;
135 
136   // user with all permissions
137   private static User SUPERUSER;
138   // user granted with all global permission
139   private static User USER_ADMIN;
140   // user with rw permissions on column family.
141   private static User USER_RW;
142   // user with read-only permissions
143   private static User USER_RO;
144   // user is table owner. will have all permissions on table
145   private static User USER_OWNER;
146   // user with create table permissions alone
147   private static User USER_CREATE;
148   // user with no permissions
149   private static User USER_NONE;
150   // user with admin rights on the column family
151   private static User USER_ADMIN_CF;
152 
153   private static final String GROUP_ADMIN = "group_admin";
154   private static final String GROUP_CREATE = "group_create";
155   private static final String GROUP_READ = "group_read";
156   private static final String GROUP_WRITE = "group_write";
157 
158   private static User USER_GROUP_ADMIN;
159   private static User USER_GROUP_CREATE;
160   private static User USER_GROUP_READ;
161   private static User USER_GROUP_WRITE;
162 
163   // TODO: convert this test to cover the full matrix in
164   // https://hbase.apache.org/book/appendix_acl_matrix.html
165   // creating all Scope x Permission combinations
166 
167   private static TableName TEST_TABLE2 = TableName.valueOf("testtable2");
168   private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
169   private static byte[] TEST_QUALIFIER = Bytes.toBytes("q1");
170   private static byte[] TEST_ROW = Bytes.toBytes("r1");
171 
172   private static MasterCoprocessorEnvironment CP_ENV;
173   private static AccessController ACCESS_CONTROLLER;
174   private static RegionServerCoprocessorEnvironment RSCP_ENV;
175   private static RegionCoprocessorEnvironment RCP_ENV;
176 
177   @BeforeClass
178   public static void setupBeforeClass() throws Exception {
179     // setup configuration
180     conf = TEST_UTIL.getConfiguration();
181     conf.set("hbase.master.hfilecleaner.plugins",
182       "org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner," +
183       "org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner");
184     conf.set("hbase.master.logcleaner.plugins",
185       "org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner");
186     // Enable security
187     enableSecurity(conf);
188     // In this particular test case, we can't use SecureBulkLoadEndpoint because its doAs will fail
189     // to move a file for a random user
190     conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
191     // Verify enableSecurity sets up what we require
192     verifyConfiguration(conf);
193 
194     // Enable EXEC permission checking
195     conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
196 
197     TEST_UTIL.startMiniCluster();
198     MasterCoprocessorHost cpHost = TEST_UTIL.getMiniHBaseCluster().getMaster().getCoprocessorHost();
199     cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
200     ACCESS_CONTROLLER = (AccessController) cpHost.findCoprocessor(AccessController.class.getName());
201     CP_ENV = cpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
202       Coprocessor.PRIORITY_HIGHEST, 1, conf);
203     RegionServerCoprocessorHost rsHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
204         .getCoprocessorHost();
205     RSCP_ENV = rsHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
206       Coprocessor.PRIORITY_HIGHEST, 1, conf);
207 
208     // Wait for the ACL table to become available
209     TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
210 
211     // create a set of test users
212     SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
213     USER_ADMIN = User.createUserForTesting(conf, "admin2", new String[0]);
214     USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
215     USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
216     USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
217     USER_CREATE = User.createUserForTesting(conf, "tbl_create", new String[0]);
218     USER_NONE = User.createUserForTesting(conf, "nouser", new String[0]);
219     USER_ADMIN_CF = User.createUserForTesting(conf, "col_family_admin", new String[0]);
220 
221     USER_GROUP_ADMIN =
222         User.createUserForTesting(conf, "user_group_admin", new String[] { GROUP_ADMIN });
223     USER_GROUP_CREATE =
224         User.createUserForTesting(conf, "user_group_create", new String[] { GROUP_CREATE });
225     USER_GROUP_READ =
226         User.createUserForTesting(conf, "user_group_read", new String[] { GROUP_READ });
227     USER_GROUP_WRITE =
228         User.createUserForTesting(conf, "user_group_write", new String[] { GROUP_WRITE });
229 
230     setUpTableAndUserPermissions();
231   }
232 
233   @AfterClass
234   public static void tearDownAfterClass() throws Exception {
235     cleanUp();
236     TEST_UTIL.shutdownMiniCluster();
237   }
238 
239   private static void setUpTableAndUserPermissions() throws Exception {
240     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
241     HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
242     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
243     hcd.setMaxVersions(100);
244     htd.addFamily(hcd);
245     htd.setOwner(USER_OWNER);
246     admin.createTable(htd, new byte[][] { Bytes.toBytes("s") });
247     TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
248 
249     HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0);
250     RegionCoprocessorHost rcpHost = region.getCoprocessorHost();
251     RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
252       Coprocessor.PRIORITY_HIGHEST, 1, conf);
253 
254     // Set up initial grants
255 
256     grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(),
257       Permission.Action.ADMIN,
258       Permission.Action.CREATE,
259       Permission.Action.READ,
260       Permission.Action.WRITE);
261 
262     grantOnTable(TEST_UTIL, USER_RW.getShortName(),
263       TEST_TABLE, TEST_FAMILY, null,
264       Permission.Action.READ,
265       Permission.Action.WRITE);
266 
267     // USER_CREATE is USER_RW plus CREATE permissions
268     grantOnTable(TEST_UTIL, USER_CREATE.getShortName(),
269       TEST_TABLE, null, null,
270       Permission.Action.CREATE,
271       Permission.Action.READ,
272       Permission.Action.WRITE);
273 
274     grantOnTable(TEST_UTIL, USER_RO.getShortName(),
275       TEST_TABLE, TEST_FAMILY, null,
276       Permission.Action.READ);
277 
278     grantOnTable(TEST_UTIL, USER_ADMIN_CF.getShortName(),
279       TEST_TABLE, TEST_FAMILY,
280       null, Permission.Action.ADMIN, Permission.Action.CREATE);
281 
282     grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN);
283     grantGlobal(TEST_UTIL, toGroupEntry(GROUP_CREATE), Permission.Action.CREATE);
284     grantGlobal(TEST_UTIL, toGroupEntry(GROUP_READ), Permission.Action.READ);
285     grantGlobal(TEST_UTIL, toGroupEntry(GROUP_WRITE), Permission.Action.WRITE);
286 
287     assertEquals(5, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
288     try {
289       assertEquals(5, AccessControlClient.getUserPermissions(conf, TEST_TABLE.toString()).size());
290     } catch (Throwable e) {
291       LOG.error("error during call of AccessControlClient.getUserPermissions. " + e.getStackTrace());
292     }
293   }
294 
295   private static void cleanUp() throws Exception {
296     // Clean the _acl_ table
297     try {
298       TEST_UTIL.deleteTable(TEST_TABLE);
299     } catch (TableNotFoundException ex) {
300       // Test deleted the table, no problem
301       LOG.info("Test deleted table " + TEST_TABLE);
302     }
303     assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE).size());
304   }
305 
306   @Test
307   public void testSecurityCapabilities() throws Exception {
308     List<SecurityCapability> capabilities = TEST_UTIL.getHBaseAdmin().getSecurityCapabilities();
309     assertTrue("AUTHORIZATION capability is missing",
310       capabilities.contains(SecurityCapability.AUTHORIZATION));
311     assertTrue("CELL_AUTHORIZATION capability is missing",
312       capabilities.contains(SecurityCapability.CELL_AUTHORIZATION));
313   }
314 
315   @Test
316   public void testTableCreate() throws Exception {
317     AccessTestAction createTable = new AccessTestAction() {
318       @Override
319       public Object run() throws Exception {
320         HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testnewtable"));
321         htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
322         ACCESS_CONTROLLER.preCreateTable(ObserverContext.createAndPrepare(CP_ENV, null), htd, null);
323         return null;
324       }
325     };
326 
327     // verify that superuser can create tables
328     verifyAllowed(createTable, SUPERUSER, USER_ADMIN, USER_GROUP_CREATE);
329 
330     // all others should be denied
331     verifyDenied(createTable, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_ADMIN,
332       USER_GROUP_READ, USER_GROUP_WRITE);
333   }
334 
335   @Test
336   public void testTableModify() throws Exception {
337     AccessTestAction modifyTable = new AccessTestAction() {
338       @Override
339       public Object run() throws Exception {
340         HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
341         htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
342         htd.addFamily(new HColumnDescriptor("fam_" + User.getCurrent().getShortName()));
343         ACCESS_CONTROLLER.preModifyTable(ObserverContext.createAndPrepare(CP_ENV, null),
344             TEST_TABLE, htd);
345         return null;
346       }
347     };
348 
349     verifyAllowed(modifyTable, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_GROUP_CREATE,
350       USER_GROUP_ADMIN);
351     verifyDenied(modifyTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
352   }
353 
354   @Test
355   public void testTableDelete() throws Exception {
356     AccessTestAction deleteTable = new AccessTestAction() {
357       @Override
358       public Object run() throws Exception {
359         ACCESS_CONTROLLER
360             .preDeleteTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
361         return null;
362       }
363     };
364 
365     verifyAllowed(deleteTable, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_GROUP_CREATE,
366       USER_GROUP_ADMIN);
367     verifyDenied(deleteTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
368   }
369 
370   @Test
371   public void testTableTruncate() throws Exception {
372     AccessTestAction truncateTable = new AccessTestAction() {
373       @Override
374       public Object run() throws Exception {
375         ACCESS_CONTROLLER
376             .preTruncateTable(ObserverContext.createAndPrepare(CP_ENV, null),
377                 TEST_TABLE);
378         return null;
379       }
380     };
381 
382     verifyAllowed(truncateTable, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_GROUP_CREATE,
383       USER_GROUP_ADMIN);
384     verifyDenied(truncateTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
385   }
386 
387   @Test
388   public void testAddColumn() throws Exception {
389     final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
390     AccessTestAction action = new AccessTestAction() {
391       @Override
392       public Object run() throws Exception {
393         ACCESS_CONTROLLER.preAddColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE,
394             hcd);
395         return null;
396       }
397     };
398 
399     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_GROUP_CREATE,
400       USER_GROUP_ADMIN);
401     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
402   }
403 
404   @Test
405   public void testModifyColumn() throws Exception {
406     final HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
407     hcd.setMaxVersions(10);
408     AccessTestAction action = new AccessTestAction() {
409       @Override
410       public Object run() throws Exception {
411         ACCESS_CONTROLLER.preModifyColumn(ObserverContext.createAndPrepare(CP_ENV, null),
412             TEST_TABLE, hcd);
413         return null;
414       }
415     };
416 
417     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_ADMIN_CF,
418       USER_GROUP_CREATE, USER_GROUP_ADMIN);
419     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
420   }
421 
422   @Test
423   public void testDeleteColumn() throws Exception {
424     AccessTestAction action = new AccessTestAction() {
425       @Override
426       public Object run() throws Exception {
427         ACCESS_CONTROLLER.preDeleteColumn(ObserverContext.createAndPrepare(CP_ENV, null),
428             TEST_TABLE, TEST_FAMILY);
429         return null;
430       }
431     };
432 
433     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_ADMIN_CF,
434         USER_GROUP_CREATE, USER_GROUP_ADMIN);
435     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
436   }
437 
438   @Test
439   public void testTableDisable() throws Exception {
440     AccessTestAction disableTable = new AccessTestAction() {
441       @Override
442       public Object run() throws Exception {
443         ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
444             TEST_TABLE);
445         return null;
446       }
447     };
448 
449     AccessTestAction disableAclTable = new AccessTestAction() {
450       @Override
451       public Object run() throws Exception {
452         ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
453             AccessControlLists.ACL_TABLE_NAME);
454         return null;
455       }
456     };
457 
458     verifyAllowed(disableTable, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_GROUP_CREATE,
459       USER_GROUP_ADMIN);
460     verifyDenied(disableTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
461 
462     // No user should be allowed to disable _acl_ table
463     verifyDenied(disableAclTable, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
464       USER_GROUP_CREATE, USER_GROUP_ADMIN, USER_GROUP_READ, USER_GROUP_WRITE);
465   }
466 
467   @Test
468   public void testTableEnable() throws Exception {
469     AccessTestAction enableTable = new AccessTestAction() {
470       @Override
471       public Object run() throws Exception {
472         ACCESS_CONTROLLER
473             .preEnableTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
474         return null;
475       }
476     };
477 
478     verifyAllowed(enableTable, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_GROUP_CREATE,
479       USER_GROUP_ADMIN);
480     verifyDenied(enableTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
481   }
482 
483   @Test
484   public void testMove() throws Exception {
485     Map<HRegionInfo, ServerName> regions;
486     HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
487     try {
488       regions = table.getRegionLocations();
489     } finally {
490       table.close();
491     }
492     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
493     final ServerName server = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
494     AccessTestAction action = new AccessTestAction() {
495       @Override
496       public Object run() throws Exception {
497         ACCESS_CONTROLLER.preMove(ObserverContext.createAndPrepare(CP_ENV, null),
498             firstRegion.getKey(), server, server);
499         return null;
500       }
501     };
502 
503     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
504     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
505       USER_GROUP_WRITE, USER_GROUP_CREATE);
506   }
507 
508   @Test
509   public void testAssign() throws Exception {
510     Map<HRegionInfo, ServerName> regions;
511     HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
512     try {
513       regions = table.getRegionLocations();
514     } finally {
515       table.close();
516     }
517     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
518 
519     AccessTestAction action = new AccessTestAction() {
520       @Override
521       public Object run() throws Exception {
522         ACCESS_CONTROLLER.preAssign(ObserverContext.createAndPrepare(CP_ENV, null),
523             firstRegion.getKey());
524         return null;
525       }
526     };
527 
528     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
529     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
530       USER_GROUP_WRITE, USER_GROUP_CREATE);
531   }
532 
533   @Test
534   public void testUnassign() throws Exception {
535     Map<HRegionInfo, ServerName> regions;
536     HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
537     try {
538       regions = table.getRegionLocations();
539     } finally {
540       table.close();
541     }
542     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
543 
544     AccessTestAction action = new AccessTestAction() {
545       @Override
546       public Object run() throws Exception {
547         ACCESS_CONTROLLER.preUnassign(ObserverContext.createAndPrepare(CP_ENV, null),
548             firstRegion.getKey(), false);
549         return null;
550       }
551     };
552 
553     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
554     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
555       USER_GROUP_WRITE, USER_GROUP_CREATE);
556   }
557 
558   @Test
559   public void testRegionOffline() throws Exception {
560     Map<HRegionInfo, ServerName> regions;
561     HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
562     try {
563       regions = table.getRegionLocations();
564     } finally {
565       table.close();
566     }
567     final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet().iterator().next();
568 
569     AccessTestAction action = new AccessTestAction() {
570       @Override
571       public Object run() throws Exception {
572         ACCESS_CONTROLLER.preRegionOffline(ObserverContext.createAndPrepare(CP_ENV, null),
573             firstRegion.getKey());
574         return null;
575       }
576     };
577 
578     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
579     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
580       USER_GROUP_WRITE, USER_GROUP_CREATE);
581   }
582 
583   @Test
584   public void testBalance() throws Exception {
585     AccessTestAction action = new AccessTestAction() {
586       @Override
587       public Object run() throws Exception {
588         ACCESS_CONTROLLER.preBalance(ObserverContext.createAndPrepare(CP_ENV, null));
589         return null;
590       }
591     };
592 
593     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
594     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
595       USER_GROUP_WRITE, USER_GROUP_CREATE);
596   }
597 
598   @Test
599   public void testBalanceSwitch() throws Exception {
600     AccessTestAction action = new AccessTestAction() {
601       @Override
602       public Object run() throws Exception {
603         ACCESS_CONTROLLER.preBalanceSwitch(ObserverContext.createAndPrepare(CP_ENV, null), true);
604         return null;
605       }
606     };
607 
608     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
609     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
610       USER_GROUP_WRITE, USER_GROUP_CREATE);
611   }
612 
613   @Test
614   public void testShutdown() throws Exception {
615     AccessTestAction action = new AccessTestAction() {
616       @Override
617       public Object run() throws Exception {
618         ACCESS_CONTROLLER.preShutdown(ObserverContext.createAndPrepare(CP_ENV, null));
619         return null;
620       }
621     };
622 
623     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
624     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
625       USER_GROUP_WRITE, USER_GROUP_CREATE);
626   }
627 
628   @Test
629   public void testStopMaster() throws Exception {
630     AccessTestAction action = new AccessTestAction() {
631       @Override
632       public Object run() throws Exception {
633         ACCESS_CONTROLLER.preStopMaster(ObserverContext.createAndPrepare(CP_ENV, null));
634         return null;
635       }
636     };
637 
638     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
639     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
640       USER_GROUP_WRITE, USER_GROUP_CREATE);
641   }
642 
643   private void verifyWrite(AccessTestAction action) throws Exception {
644     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW,
645       USER_GROUP_WRITE);
646     verifyDenied(action, USER_NONE, USER_RO, USER_GROUP_ADMIN, USER_GROUP_READ, USER_GROUP_CREATE);
647   }
648 
649   @Test
650   public void testSplit() throws Exception {
651     AccessTestAction action = new AccessTestAction() {
652       @Override
653       public Object run() throws Exception {
654         ACCESS_CONTROLLER.preSplit(ObserverContext.createAndPrepare(RCP_ENV, null));
655         return null;
656       }
657     };
658 
659     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
660     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
661       USER_GROUP_WRITE, USER_GROUP_CREATE);
662   }
663 
664   @Test
665   public void testSplitWithSplitRow() throws Exception {
666     AccessTestAction action = new AccessTestAction() {
667       @Override
668       public Object run() throws Exception {
669         ACCESS_CONTROLLER.preSplit(
670             ObserverContext.createAndPrepare(RCP_ENV, null),
671             TEST_ROW);
672         return null;
673       }
674     };
675 
676     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
677     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
678         USER_GROUP_WRITE, USER_GROUP_CREATE);
679   }
680 
681   @Test
682   public void testMergeRegions() throws Exception {
683     final TableName tname = TableName.valueOf("testMergeRegions");
684     createTestTable(tname);
685     try {
686       final List<HRegion> regions = TEST_UTIL.getHBaseCluster().findRegionsForTable(TEST_TABLE);
687 
688       AccessTestAction action = new AccessTestAction() {
689         @Override
690         public Object run() throws Exception {
691           ACCESS_CONTROLLER.preMerge(ObserverContext.createAndPrepare(RSCP_ENV, null),
692             regions.get(0), regions.get(1));
693           return null;
694         }
695       };
696 
697       verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
698       verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
699         USER_GROUP_WRITE, USER_GROUP_CREATE);
700     } finally {
701       TEST_UTIL.deleteTable(tname);
702     }
703   }
704 
705   private void createTestTable(TableName tname) throws Exception {
706     HTableDescriptor htd = new HTableDescriptor(tname);
707     HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
708     hcd.setMaxVersions(100);
709     htd.addFamily(hcd);
710     htd.setOwner(USER_OWNER);
711     TEST_UTIL.createTable(htd, new byte[][] { Bytes.toBytes("s") });
712   }
713 
714   @Test
715   public void testFlush() throws Exception {
716     AccessTestAction action = new AccessTestAction() {
717       @Override
718       public Object run() throws Exception {
719         ACCESS_CONTROLLER.preFlush(ObserverContext.createAndPrepare(RCP_ENV, null));
720         return null;
721       }
722     };
723 
724     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_GROUP_CREATE,
725       USER_GROUP_ADMIN);
726     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
727   }
728 
729   @Test
730   public void testCompact() throws Exception {
731     AccessTestAction action = new AccessTestAction() {
732       @Override
733       public Object run() throws Exception {
734         ACCESS_CONTROLLER.preCompact(ObserverContext.createAndPrepare(RCP_ENV, null), null, null,
735             ScanType.COMPACT_RETAIN_DELETES);
736         return null;
737       }
738     };
739 
740     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_GROUP_CREATE,
741       USER_GROUP_ADMIN);
742     verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
743   }
744 
745   private void verifyRead(AccessTestAction action) throws Exception {
746     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW, USER_RO,
747       USER_GROUP_READ);
748     verifyDenied(action, USER_NONE, USER_GROUP_CREATE, USER_GROUP_ADMIN, USER_GROUP_WRITE);
749   }
750 
751   private void verifyReadWrite(AccessTestAction action) throws Exception {
752     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW);
753     verifyDenied(action, USER_NONE, USER_RO, USER_GROUP_ADMIN, USER_GROUP_CREATE, USER_GROUP_READ,
754         USER_GROUP_WRITE);
755   }
756 
757   @Test
758   public void testRead() throws Exception {
759     // get action
760     AccessTestAction getAction = new AccessTestAction() {
761       @Override
762       public Object run() throws Exception {
763         Get g = new Get(TEST_ROW);
764         g.addFamily(TEST_FAMILY);
765 
766         HTable t = new HTable(conf, TEST_TABLE);
767         try {
768           t.get(g);
769         } finally {
770           t.close();
771         }
772         return null;
773       }
774     };
775     verifyRead(getAction);
776 
777     // action for scanning
778     AccessTestAction scanAction = new AccessTestAction() {
779       @Override
780       public Object run() throws Exception {
781         Scan s = new Scan();
782         s.addFamily(TEST_FAMILY);
783 
784         HTable table = new HTable(conf, TEST_TABLE);
785         try {
786           ResultScanner scanner = table.getScanner(s);
787           try {
788             for (Result r = scanner.next(); r != null; r = scanner.next()) {
789               // do nothing
790             }
791           } catch (IOException e) {
792           } finally {
793             scanner.close();
794           }
795         } finally {
796           table.close();
797         }
798         return null;
799       }
800     };
801     verifyRead(scanAction);
802   }
803 
804   @Test
805   // test put, delete, increment
806   public void testWrite() throws Exception {
807     // put action
808     AccessTestAction putAction = new AccessTestAction() {
809       @Override
810       public Object run() throws Exception {
811         Put p = new Put(TEST_ROW);
812         p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1));
813         HTable t = new HTable(conf, TEST_TABLE);
814         try {
815           t.put(p);
816         } finally {
817           t.close();
818         }
819         return null;
820       }
821     };
822     verifyWrite(putAction);
823 
824     // delete action
825     AccessTestAction deleteAction = new AccessTestAction() {
826       @Override
827       public Object run() throws Exception {
828         Delete d = new Delete(TEST_ROW);
829         d.deleteFamily(TEST_FAMILY);
830         HTable t = new HTable(conf, TEST_TABLE);
831         try {
832           t.delete(d);
833         } finally {
834           t.close();
835         }
836         return null;
837       }
838     };
839     verifyWrite(deleteAction);
840 
841     // increment action
842     AccessTestAction incrementAction = new AccessTestAction() {
843       @Override
844       public Object run() throws Exception {
845         Increment inc = new Increment(TEST_ROW);
846         inc.addColumn(TEST_FAMILY, TEST_QUALIFIER, 1);
847         HTable t = new HTable(conf, TEST_TABLE);
848         try {
849           t.increment(inc);
850         } finally {
851           t.close();
852         }
853         return null;
854       }
855     };
856     verifyWrite(incrementAction);
857   }
858 
859   @Test
860   public void testReadWrite() throws Exception {
861     // action for checkAndDelete
862     AccessTestAction checkAndDeleteAction = new AccessTestAction() {
863       @Override
864       public Object run() throws Exception {
865         Delete d = new Delete(TEST_ROW);
866         d.deleteFamily(TEST_FAMILY);
867         HTable t = new HTable(conf, TEST_TABLE);
868         try {
869           t.checkAndDelete(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
870             Bytes.toBytes("test_value"), d);
871         } finally {
872           t.close();
873         }
874         return null;
875       }
876     };
877     verifyReadWrite(checkAndDeleteAction);
878 
879     // action for checkAndPut()
880     AccessTestAction checkAndPut = new AccessTestAction() {
881       @Override
882       public Object run() throws Exception {
883         Put p = new Put(TEST_ROW);
884         p.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(1));
885         HTable t = new HTable(conf, TEST_TABLE);
886         try {
887           t.checkAndPut(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
888            Bytes.toBytes("test_value"), p);
889         } finally {
890           t.close();
891         }
892         return null;
893       }
894     };
895     verifyReadWrite(checkAndPut);
896   }
897 
898   @Test
899   public void testBulkLoad() throws Exception {
900     try {
901       FileSystem fs = TEST_UTIL.getTestFileSystem();
902       final Path dir = TEST_UTIL.getDataTestDirOnTestFS("testBulkLoad");
903       fs.mkdirs(dir);
904       // need to make it globally writable
905       // so users creating HFiles have write permissions
906       fs.setPermission(dir, FsPermission.valueOf("-rwxrwxrwx"));
907 
908       AccessTestAction bulkLoadAction = new AccessTestAction() {
909         @Override
910         public Object run() throws Exception {
911           int numRows = 3;
912 
913           // Making the assumption that the test table won't split between the range
914           byte[][][] hfileRanges = { { { (byte) 0 }, { (byte) 9 } } };
915 
916           Path bulkLoadBasePath = new Path(dir, new Path(User.getCurrent().getName()));
917           new BulkLoadHelper(bulkLoadBasePath).bulkLoadHFile(TEST_TABLE, TEST_FAMILY,
918             TEST_QUALIFIER, hfileRanges, numRows);
919 
920           return null;
921         }
922       };
923 
924       // User performing bulk loads must have privilege to read table metadata
925       // (ADMIN or CREATE)
926       verifyAllowed(bulkLoadAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE,
927         USER_GROUP_CREATE);
928       verifyDenied(bulkLoadAction, USER_RW, USER_NONE, USER_RO, USER_GROUP_READ, USER_GROUP_WRITE,
929         USER_GROUP_ADMIN);
930 
931     } finally {
932       // Reinit after the bulk upload
933       TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE);
934       TEST_UTIL.getHBaseAdmin().enableTable(TEST_TABLE);
935     }
936   }
937 
938   public class BulkLoadHelper {
939     private final FileSystem fs;
940     private final Path loadPath;
941     private final Configuration conf;
942 
943     public BulkLoadHelper(Path loadPath) throws IOException {
944       fs = TEST_UTIL.getTestFileSystem();
945       conf = TEST_UTIL.getConfiguration();
946       loadPath = loadPath.makeQualified(fs);
947       this.loadPath = loadPath;
948     }
949 
950     private void createHFile(Path path,
951         byte[] family, byte[] qualifier,
952         byte[] startKey, byte[] endKey, int numRows) throws IOException {
953 
954       HFile.Writer writer = null;
955       long now = System.currentTimeMillis();
956       try {
957         HFileContext context = new HFileContextBuilder().build();
958         writer = HFile.getWriterFactory(conf, new CacheConfig(conf))
959             .withPath(fs, path)
960             .withFileContext(context)
961             .create();
962         // subtract 2 since numRows doesn't include boundary keys
963         for (byte[] key : Bytes.iterateOnSplits(startKey, endKey, true, numRows-2)) {
964           KeyValue kv = new KeyValue(key, family, qualifier, now, key);
965           writer.append(kv);
966         }
967       } finally {
968         if(writer != null)
969           writer.close();
970       }
971     }
972 
973     private void bulkLoadHFile(
974         TableName tableName,
975         byte[] family,
976         byte[] qualifier,
977         byte[][][] hfileRanges,
978         int numRowsPerRange) throws Exception {
979 
980       Path familyDir = new Path(loadPath, Bytes.toString(family));
981       fs.mkdirs(familyDir);
982       int hfileIdx = 0;
983       for (byte[][] range : hfileRanges) {
984         byte[] from = range[0];
985         byte[] to = range[1];
986         createHFile(new Path(familyDir, "hfile_"+(hfileIdx++)),
987             family, qualifier, from, to, numRowsPerRange);
988       }
989       //set global read so RegionServer can move it
990       setPermission(loadPath, FsPermission.valueOf("-rwxrwxrwx"));
991 
992       HTable table = new HTable(conf, tableName);
993       try {
994         HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
995         TEST_UTIL.waitTableEnabled(admin, tableName.getName());
996         LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
997         loader.doBulkLoad(loadPath, table);
998       } finally {
999         table.close();
1000       }
1001     }
1002 
1003     public void setPermission(Path dir, FsPermission perm) throws IOException {
1004       if(!fs.getFileStatus(dir).isDir()) {
1005         fs.setPermission(dir,perm);
1006       }
1007       else {
1008         for(FileStatus el : fs.listStatus(dir)) {
1009           fs.setPermission(el.getPath(), perm);
1010           setPermission(el.getPath() , perm);
1011         }
1012       }
1013     }
1014   }
1015 
1016   @Test
1017   public void testAppend() throws Exception {
1018 
1019     AccessTestAction appendAction = new AccessTestAction() {
1020       @Override
1021       public Object run() throws Exception {
1022         byte[] row = TEST_ROW;
1023         byte[] qualifier = TEST_QUALIFIER;
1024         Put put = new Put(row);
1025         put.add(TEST_FAMILY, qualifier, Bytes.toBytes(1));
1026         Append append = new Append(row);
1027         append.add(TEST_FAMILY, qualifier, Bytes.toBytes(2));
1028         HTable t = new HTable(conf, TEST_TABLE);
1029         try {
1030           t.put(put);
1031           t.append(append);
1032         } finally {
1033           t.close();
1034         }
1035         return null;
1036       }
1037     };
1038 
1039     verifyAllowed(appendAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_RW,
1040       USER_GROUP_WRITE);
1041     verifyDenied(appendAction, USER_RO, USER_NONE, USER_GROUP_CREATE, USER_GROUP_READ,
1042       USER_GROUP_ADMIN);
1043   }
1044 
1045   @Test
1046   public void testGrantRevoke() throws Exception {
1047     AccessTestAction grantAction = new AccessTestAction() {
1048       @Override
1049       public Object run() throws Exception {
1050         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1051         try {
1052           BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
1053           AccessControlService.BlockingInterface protocol =
1054             AccessControlService.newBlockingStub(service);
1055           ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE,
1056             TEST_FAMILY, null, Action.READ);
1057         } finally {
1058           acl.close();
1059         }
1060         return null;
1061       }
1062     };
1063 
1064     AccessTestAction revokeAction = new AccessTestAction() {
1065       @Override
1066       public Object run() throws Exception {
1067         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1068         try {
1069           BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
1070           AccessControlService.BlockingInterface protocol =
1071             AccessControlService.newBlockingStub(service);
1072           ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE,
1073             TEST_FAMILY, null, Action.READ);
1074         } finally {
1075           acl.close();
1076         }
1077         return null;
1078       }
1079     };
1080 
1081     AccessTestAction getTablePermissionsAction = new AccessTestAction() {
1082       @Override
1083       public Object run() throws Exception {
1084         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1085         try {
1086           BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
1087           AccessControlService.BlockingInterface protocol =
1088             AccessControlService.newBlockingStub(service);
1089           ProtobufUtil.getUserPermissions(protocol, TEST_TABLE);
1090         } finally {
1091           acl.close();
1092         }
1093         return null;
1094       }
1095     };
1096 
1097     AccessTestAction getGlobalPermissionsAction = new AccessTestAction() {
1098       @Override
1099       public Object run() throws Exception {
1100         HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1101         try {
1102           BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
1103           AccessControlService.BlockingInterface protocol =
1104             AccessControlService.newBlockingStub(service);
1105           ProtobufUtil.getUserPermissions(protocol);
1106         } finally {
1107           acl.close();
1108         }
1109         return null;
1110       }
1111     };
1112 
1113     verifyAllowed(grantAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
1114     verifyDenied(grantAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
1115         USER_GROUP_WRITE, USER_GROUP_CREATE);
1116     try {
1117       verifyAllowed(revokeAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
1118       verifyDenied(revokeAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
1119         USER_GROUP_WRITE, USER_GROUP_CREATE);
1120 
1121       verifyAllowed(getTablePermissionsAction, SUPERUSER, USER_ADMIN, USER_OWNER, USER_GROUP_ADMIN);
1122       verifyDenied(getTablePermissionsAction, USER_CREATE, USER_RW, USER_RO, USER_NONE,
1123         USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
1124 
1125       verifyAllowed(getGlobalPermissionsAction, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
1126       verifyDenied(getGlobalPermissionsAction, USER_CREATE, USER_OWNER, USER_RW, USER_RO,
1127         USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
1128     } finally {
1129       // Cleanup, Grant the revoked permission back to the user
1130       grantOnTable(TEST_UTIL, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
1131         Permission.Action.READ);
1132     }
1133   }
1134 
1135   @Test
1136   public void testPostGrantRevoke() throws Exception {
1137     final TableName tableName =
1138         TableName.valueOf("TempTable");
1139     final byte[] family1 = Bytes.toBytes("f1");
1140     final byte[] family2 = Bytes.toBytes("f2");
1141     final byte[] qualifier = Bytes.toBytes("q");
1142 
1143     // create table
1144     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
1145     if (admin.tableExists(tableName)) {
1146       admin.disableTable(tableName);
1147       admin.deleteTable(tableName);
1148     }
1149     HTableDescriptor htd = new HTableDescriptor(tableName);
1150     htd.addFamily(new HColumnDescriptor(family1));
1151     htd.addFamily(new HColumnDescriptor(family2));
1152     admin.createTable(htd);
1153     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
1154     try {
1155       // create temp users
1156       User tblUser =
1157           User.createUserForTesting(TEST_UTIL.getConfiguration(), "tbluser", new String[0]);
1158       User gblUser =
1159           User.createUserForTesting(TEST_UTIL.getConfiguration(), "gbluser", new String[0]);
1160 
1161       // prepare actions:
1162       AccessTestAction putActionAll = new AccessTestAction() {
1163         @Override
1164         public Object run() throws Exception {
1165           Put p = new Put(Bytes.toBytes("a"));
1166           p.add(family1, qualifier, Bytes.toBytes("v1"));
1167           p.add(family2, qualifier, Bytes.toBytes("v2"));
1168           HTable t = new HTable(conf, tableName);
1169           try {
1170             t.put(p);
1171           } finally {
1172             t.close();
1173           }
1174           return null;
1175         }
1176       };
1177 
1178       AccessTestAction putAction1 = new AccessTestAction() {
1179         @Override
1180         public Object run() throws Exception {
1181           Put p = new Put(Bytes.toBytes("a"));
1182           p.add(family1, qualifier, Bytes.toBytes("v1"));
1183           HTable t = new HTable(conf, tableName);
1184           try {
1185             t.put(p);
1186           } finally {
1187             t.close();
1188           }
1189           return null;
1190         }
1191       };
1192 
1193       AccessTestAction putAction2 = new AccessTestAction() {
1194         @Override
1195         public Object run() throws Exception {
1196           Put p = new Put(Bytes.toBytes("a"));
1197           p.add(family2, qualifier, Bytes.toBytes("v2"));
1198           HTable t = new HTable(conf, tableName);
1199           try {
1200             t.put(p);
1201           } finally {
1202             t.close();
1203           }
1204           return null;
1205         }
1206       };
1207 
1208       AccessTestAction getActionAll = new AccessTestAction() {
1209         @Override
1210         public Object run() throws Exception {
1211           Get g = new Get(TEST_ROW);
1212           g.addFamily(family1);
1213           g.addFamily(family2);
1214           HTable t = new HTable(conf, tableName);
1215           try {
1216             t.get(g);
1217           } finally {
1218             t.close();
1219           }
1220           return null;
1221         }
1222       };
1223 
1224       AccessTestAction getAction1 = new AccessTestAction() {
1225         @Override
1226         public Object run() throws Exception {
1227           Get g = new Get(TEST_ROW);
1228           g.addFamily(family1);
1229           HTable t = new HTable(conf, tableName);
1230           try {
1231             t.get(g);
1232           } finally {
1233             t.close();
1234           }
1235           return null;
1236         }
1237       };
1238 
1239       AccessTestAction getAction2 = new AccessTestAction() {
1240         @Override
1241         public Object run() throws Exception {
1242           Get g = new Get(TEST_ROW);
1243           g.addFamily(family2);
1244           HTable t = new HTable(conf, tableName);
1245           try {
1246             t.get(g);
1247           } finally {
1248             t.close();
1249           }
1250           return null;
1251         }
1252       };
1253 
1254       AccessTestAction deleteActionAll = new AccessTestAction() {
1255         @Override
1256         public Object run() throws Exception {
1257           Delete d = new Delete(TEST_ROW);
1258           d.deleteFamily(family1);
1259           d.deleteFamily(family2);
1260           HTable t = new HTable(conf, tableName);
1261           try {
1262             t.delete(d);
1263           } finally {
1264             t.close();
1265           }
1266           return null;
1267         }
1268       };
1269 
1270       AccessTestAction deleteAction1 = new AccessTestAction() {
1271         @Override
1272         public Object run() throws Exception {
1273           Delete d = new Delete(TEST_ROW);
1274           d.deleteFamily(family1);
1275           HTable t = new HTable(conf, tableName);
1276           try {
1277             t.delete(d);
1278           } finally {
1279             t.close();
1280           }
1281           return null;
1282         }
1283       };
1284 
1285       AccessTestAction deleteAction2 = new AccessTestAction() {
1286         @Override
1287         public Object run() throws Exception {
1288           Delete d = new Delete(TEST_ROW);
1289           d.deleteFamily(family2);
1290           HTable t = new HTable(conf, tableName);
1291           try {
1292             t.delete(d);
1293           } finally {
1294             t.close();
1295           }
1296           return null;
1297         }
1298       };
1299 
1300       // initial check:
1301       verifyDenied(tblUser, getActionAll, getAction1, getAction2);
1302       verifyDenied(tblUser, putActionAll, putAction1, putAction2);
1303       verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
1304 
1305       verifyDenied(gblUser, getActionAll, getAction1, getAction2);
1306       verifyDenied(gblUser, putActionAll, putAction1, putAction2);
1307       verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
1308 
1309       // grant table read permission
1310       grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.READ);
1311       grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, null, null,
1312         Permission.Action.READ);
1313 
1314       // check
1315       verifyAllowed(tblUser, getActionAll, getAction1, getAction2);
1316       verifyDenied(tblUser, putActionAll, putAction1, putAction2);
1317       verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
1318 
1319       verifyAllowed(gblUser, getActionAll, getAction1, getAction2);
1320       verifyDenied(gblUser, putActionAll, putAction1, putAction2);
1321       verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
1322 
1323       // grant table write permission while revoking read permissions
1324       grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.WRITE);
1325       grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, null, null,
1326         Permission.Action.WRITE);
1327 
1328       verifyDenied(tblUser, getActionAll, getAction1, getAction2);
1329       verifyAllowed(tblUser, putActionAll, putAction1, putAction2);
1330       verifyAllowed(tblUser, deleteActionAll, deleteAction1, deleteAction2);
1331 
1332       verifyDenied(gblUser, getActionAll, getAction1, getAction2);
1333       verifyAllowed(gblUser, putActionAll, putAction1, putAction2);
1334       verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
1335 
1336       // revoke table permissions
1337       revokeGlobal(TEST_UTIL, gblUser.getShortName());
1338       revokeFromTable(TEST_UTIL, tblUser.getShortName(), tableName, null, null);
1339 
1340       verifyDenied(tblUser, getActionAll, getAction1, getAction2);
1341       verifyDenied(tblUser, putActionAll, putAction1, putAction2);
1342       verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
1343 
1344       verifyDenied(gblUser, getActionAll, getAction1, getAction2);
1345       verifyDenied(gblUser, putActionAll, putAction1, putAction2);
1346       verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
1347 
1348       // grant column family read permission
1349       grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.READ);
1350       grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, family1, null,
1351         Permission.Action.READ);
1352 
1353       // Access should be denied for family2
1354       verifyAllowed(tblUser, getActionAll, getAction1);
1355       verifyDenied(tblUser, getAction2);
1356       verifyDenied(tblUser, putActionAll, putAction1, putAction2);
1357       verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
1358 
1359       verifyAllowed(gblUser, getActionAll, getAction1, getAction2);
1360       verifyDenied(gblUser, putActionAll, putAction1, putAction2);
1361       verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
1362 
1363       // grant column family write permission
1364       grantGlobal(TEST_UTIL, gblUser.getShortName(), Permission.Action.WRITE);
1365       grantOnTable(TEST_UTIL, tblUser.getShortName(), tableName, family2, null,
1366         Permission.Action.WRITE);
1367 
1368       // READ from family1, WRITE to family2 are allowed
1369       verifyAllowed(tblUser, getActionAll, getAction1);
1370       verifyAllowed(tblUser, putAction2, deleteAction2);
1371       verifyDenied(tblUser, getAction2);
1372       verifyDenied(tblUser, putActionAll, putAction1);
1373       verifyDenied(tblUser, deleteActionAll, deleteAction1);
1374 
1375       verifyDenied(gblUser, getActionAll, getAction1, getAction2);
1376       verifyAllowed(gblUser, putActionAll, putAction1, putAction2);
1377       verifyAllowed(gblUser, deleteActionAll, deleteAction1, deleteAction2);
1378 
1379       // revoke column family permission
1380       revokeGlobal(TEST_UTIL, gblUser.getShortName());
1381       revokeFromTable(TEST_UTIL, tblUser.getShortName(), tableName, family2, null);
1382 
1383       // Revoke on family2 should not have impact on family1 permissions
1384       verifyAllowed(tblUser, getActionAll, getAction1);
1385       verifyDenied(tblUser, getAction2);
1386       verifyDenied(tblUser, putActionAll, putAction1, putAction2);
1387       verifyDenied(tblUser, deleteActionAll, deleteAction1, deleteAction2);
1388 
1389       // Should not have access as global permissions are completely revoked
1390       verifyDenied(gblUser, getActionAll, getAction1, getAction2);
1391       verifyDenied(gblUser, putActionAll, putAction1, putAction2);
1392       verifyDenied(gblUser, deleteActionAll, deleteAction1, deleteAction2);
1393 
1394       // delete table
1395       admin.disableTable(tableName);
1396       admin.deleteTable(tableName);
1397     } finally {
1398       try {
1399         TEST_UTIL.deleteTable(tableName);
1400       } catch (IOException ignore) {
1401         LOG.debug("Failed to delete table in cleanup. May be already deleted.");
1402       }
1403     }
1404   }
1405 
1406   private boolean hasFoundUserPermission(List<UserPermission> userPermissions,
1407                                          List<UserPermission> perms) {
1408     return perms.containsAll(userPermissions);
1409   }
1410 
1411   private boolean hasFoundUserPermission(UserPermission userPermission, List<UserPermission> perms) {
1412     return perms.contains(userPermission);
1413   }
1414 
1415   @Test
1416   public void testPostGrantRevokeAtQualifierLevel() throws Exception {
1417     final TableName tableName =
1418         TableName.valueOf("testGrantRevokeAtQualifierLevel");
1419     final byte[] family1 = Bytes.toBytes("f1");
1420     final byte[] family2 = Bytes.toBytes("f2");
1421     final byte[] qualifier = Bytes.toBytes("q");
1422 
1423     // create table
1424     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
1425     if (admin.tableExists(tableName)) {
1426       admin.disableTable(tableName);
1427       admin.deleteTable(tableName);
1428     }
1429     HTableDescriptor htd = new HTableDescriptor(tableName);
1430     htd.addFamily(new HColumnDescriptor(family1));
1431     htd.addFamily(new HColumnDescriptor(family2));
1432     admin.createTable(htd);
1433     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
1434     try {
1435       // create temp users
1436       User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "user", new String[0]);
1437 
1438       AccessTestAction getQualifierAction = new AccessTestAction() {
1439         @Override
1440         public Object run() throws Exception {
1441           Get g = new Get(TEST_ROW);
1442           g.addColumn(family1, qualifier);
1443           HTable t = new HTable(conf, tableName);
1444           try {
1445             t.get(g);
1446           } finally {
1447             t.close();
1448           }
1449           return null;
1450         }
1451       };
1452 
1453       AccessTestAction putQualifierAction = new AccessTestAction() {
1454         @Override
1455         public Object run() throws Exception {
1456           Put p = new Put(TEST_ROW);
1457           p.add(family1, qualifier, Bytes.toBytes("v1"));
1458           HTable t = new HTable(conf, tableName);
1459           try {
1460             t.put(p);
1461           } finally {
1462             t.close();
1463           }
1464           return null;
1465         }
1466       };
1467 
1468       AccessTestAction deleteQualifierAction = new AccessTestAction() {
1469         @Override
1470         public Object run() throws Exception {
1471           Delete d = new Delete(TEST_ROW);
1472           d.deleteColumn(family1, qualifier);
1473           // d.deleteFamily(family1);
1474           HTable t = new HTable(conf, tableName);
1475           try {
1476             t.delete(d);
1477           } finally {
1478             t.close();
1479           }
1480           return null;
1481         }
1482       };
1483 
1484       revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, null);
1485 
1486       verifyDenied(user, getQualifierAction);
1487       verifyDenied(user, putQualifierAction);
1488       verifyDenied(user, deleteQualifierAction);
1489 
1490       grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
1491         Permission.Action.READ);
1492 
1493       verifyAllowed(user, getQualifierAction);
1494       verifyDenied(user, putQualifierAction);
1495       verifyDenied(user, deleteQualifierAction);
1496 
1497       // only grant write permission
1498       // TODO: comment this portion after HBASE-3583
1499       grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
1500         Permission.Action.WRITE);
1501 
1502       verifyDenied(user, getQualifierAction);
1503       verifyAllowed(user, putQualifierAction);
1504       verifyAllowed(user, deleteQualifierAction);
1505 
1506       // grant both read and write permission
1507       grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
1508         Permission.Action.READ, Permission.Action.WRITE);
1509 
1510       verifyAllowed(user, getQualifierAction);
1511       verifyAllowed(user, putQualifierAction);
1512       verifyAllowed(user, deleteQualifierAction);
1513 
1514       // revoke family level permission won't impact column level
1515       revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier);
1516 
1517       verifyDenied(user, getQualifierAction);
1518       verifyDenied(user, putQualifierAction);
1519       verifyDenied(user, deleteQualifierAction);
1520 
1521       // delete table
1522       admin.disableTable(tableName);
1523       admin.deleteTable(tableName);
1524     } finally {
1525       // delete table
1526       try {
1527         TEST_UTIL.deleteTable(tableName);
1528       } catch (IOException ignore) {
1529         LOG.debug("Failed to delete table in cleanup. May be already deleted.");
1530       }
1531     }
1532   }
1533 
1534   @Test
1535   public void testPermissionList() throws Exception {
1536     final TableName tableName =
1537         TableName.valueOf("testPermissionList");
1538     final byte[] family1 = Bytes.toBytes("f1");
1539     final byte[] family2 = Bytes.toBytes("f2");
1540     final byte[] qualifier = Bytes.toBytes("q");
1541 
1542     // create table
1543     HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
1544     if (admin.tableExists(tableName)) {
1545       admin.disableTable(tableName);
1546       admin.deleteTable(tableName);
1547     }
1548     HTableDescriptor htd = new HTableDescriptor(tableName);
1549     htd.addFamily(new HColumnDescriptor(family1));
1550     htd.addFamily(new HColumnDescriptor(family2));
1551     htd.setOwner(USER_OWNER);
1552     admin.createTable(htd);
1553     TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
1554 
1555     List<UserPermission> perms;
1556     try {
1557       HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1558       try {
1559         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
1560         AccessControlService.BlockingInterface protocol =
1561             AccessControlService.newBlockingStub(service);
1562         perms = ProtobufUtil.getUserPermissions(protocol, tableName);
1563       } finally {
1564         acl.close();
1565       }
1566 
1567       UserPermission ownerperm =
1568           new UserPermission(Bytes.toBytes(USER_OWNER.getName()), tableName, null, Action.values());
1569       assertTrue("Owner should have all permissions on table",
1570         hasFoundUserPermission(ownerperm, perms));
1571 
1572       User user = User.createUserForTesting(TEST_UTIL.getConfiguration(), "user", new String[0]);
1573       byte[] userName = Bytes.toBytes(user.getShortName());
1574 
1575       UserPermission up =
1576           new UserPermission(userName, tableName, family1, qualifier, Permission.Action.READ);
1577       assertFalse("User should not be granted permission: " + up.toString(),
1578         hasFoundUserPermission(up, perms));
1579 
1580       // grant read permission
1581       grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
1582         Permission.Action.READ);
1583 
1584       acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1585       try {
1586         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
1587         AccessControlService.BlockingInterface protocol =
1588             AccessControlService.newBlockingStub(service);
1589         perms = ProtobufUtil.getUserPermissions(protocol, tableName);
1590       } finally {
1591         acl.close();
1592       }
1593 
1594       UserPermission upToVerify =
1595           new UserPermission(userName, tableName, family1, qualifier, Permission.Action.READ);
1596       assertTrue("User should be granted permission: " + upToVerify.toString(),
1597         hasFoundUserPermission(upToVerify, perms));
1598 
1599       upToVerify =
1600           new UserPermission(userName, tableName, family1, qualifier, Permission.Action.WRITE);
1601       assertFalse("User should not be granted permission: " + upToVerify.toString(),
1602         hasFoundUserPermission(upToVerify, perms));
1603 
1604       // grant read+write
1605       grantOnTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
1606         Permission.Action.WRITE, Permission.Action.READ);
1607 
1608       acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1609       try {
1610         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
1611         AccessControlService.BlockingInterface protocol =
1612             AccessControlService.newBlockingStub(service);
1613         perms = ProtobufUtil.getUserPermissions(protocol, tableName);
1614       } finally {
1615         acl.close();
1616       }
1617 
1618       upToVerify =
1619           new UserPermission(userName, tableName, family1, qualifier, Permission.Action.WRITE,
1620               Permission.Action.READ);
1621       assertTrue("User should be granted permission: " + upToVerify.toString(),
1622         hasFoundUserPermission(upToVerify, perms));
1623 
1624       // revoke
1625       revokeFromTable(TEST_UTIL, user.getShortName(), tableName, family1, qualifier,
1626         Permission.Action.WRITE, Permission.Action.READ);
1627 
1628       acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1629       try {
1630         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
1631         AccessControlService.BlockingInterface protocol =
1632             AccessControlService.newBlockingStub(service);
1633         perms = ProtobufUtil.getUserPermissions(protocol, tableName);
1634       } finally {
1635         acl.close();
1636       }
1637 
1638       assertFalse("User should not be granted permission: " + upToVerify.toString(),
1639         hasFoundUserPermission(upToVerify, perms));
1640 
1641       // disable table before modification
1642       admin.disableTable(tableName);
1643 
1644       User newOwner = User.createUserForTesting(conf, "new_owner", new String[] {});
1645       htd.setOwner(newOwner);
1646       admin.modifyTable(tableName, htd);
1647 
1648       acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1649       try {
1650         BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
1651         AccessControlService.BlockingInterface protocol =
1652             AccessControlService.newBlockingStub(service);
1653         perms = ProtobufUtil.getUserPermissions(protocol, tableName);
1654       } finally {
1655         acl.close();
1656       }
1657 
1658       UserPermission newOwnerperm =
1659           new UserPermission(Bytes.toBytes(newOwner.getName()), tableName, null, Action.values());
1660       assertTrue("New owner should have all permissions on table",
1661         hasFoundUserPermission(newOwnerperm, perms));
1662 
1663     } finally {
1664       // delete table
1665       TEST_UTIL.deleteTable(tableName);
1666     }
1667   }
1668 
1669   @Test
1670   public void testGlobalPermissionList() throws Exception {
1671     List<UserPermission> perms;
1672     HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1673     try {
1674       BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
1675       AccessControlService.BlockingInterface protocol =
1676         AccessControlService.newBlockingStub(service);
1677       perms = ProtobufUtil.getUserPermissions(protocol);
1678     } finally {
1679       acl.close();
1680     }
1681     List<UserPermission> adminPerms = new ArrayList<UserPermission>();
1682     adminPerms.add(new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
1683       AccessControlLists.ACL_TABLE_NAME, null, null, Bytes.toBytes("ACRW")));
1684     List<String> superUsers = Superusers.getSuperUsers();
1685     for(String user: superUsers) {
1686       adminPerms.add(new UserPermission(Bytes.toBytes(user), AccessControlLists.ACL_TABLE_NAME,
1687           null, null, Action.values()));
1688     }
1689     assertTrue("Only super users, global users and user admin has permission on table hbase:acl " +
1690         "per setup", perms.size() == 5 + superUsers.size() &&
1691         hasFoundUserPermission(adminPerms, perms));
1692   }
1693 
1694   /** global operations */
1695   private void verifyGlobal(AccessTestAction action) throws Exception {
1696     verifyAllowed(action, SUPERUSER);
1697 
1698     verifyDenied(action, USER_CREATE, USER_RW, USER_NONE, USER_RO);
1699   }
1700 
1701   @Test
1702   public void testCheckPermissions() throws Exception {
1703     // --------------------------------------
1704     // test global permissions
1705     AccessTestAction globalAdmin = new AccessTestAction() {
1706       @Override
1707       public Void run() throws Exception {
1708         checkGlobalPerms(TEST_UTIL, Permission.Action.ADMIN);
1709         return null;
1710       }
1711     };
1712     // verify that only superuser can admin
1713     verifyGlobal(globalAdmin);
1714 
1715     // --------------------------------------
1716     // test multiple permissions
1717     AccessTestAction globalReadWrite = new AccessTestAction() {
1718       @Override
1719       public Void run() throws Exception {
1720         checkGlobalPerms(TEST_UTIL, Permission.Action.READ, Permission.Action.WRITE);
1721         return null;
1722       }
1723     };
1724 
1725     verifyGlobal(globalReadWrite);
1726 
1727     // --------------------------------------
1728     // table/column/qualifier level permissions
1729     final byte[] TEST_Q1 = Bytes.toBytes("q1");
1730     final byte[] TEST_Q2 = Bytes.toBytes("q2");
1731 
1732     User userTable = User.createUserForTesting(conf, "user_check_perms_table", new String[0]);
1733     User userColumn = User.createUserForTesting(conf, "user_check_perms_family", new String[0]);
1734     User userQualifier = User.createUserForTesting(conf, "user_check_perms_q", new String[0]);
1735 
1736     grantOnTable(TEST_UTIL, userTable.getShortName(),
1737       TEST_TABLE, null, null,
1738       Permission.Action.READ);
1739     grantOnTable(TEST_UTIL, userColumn.getShortName(),
1740       TEST_TABLE, TEST_FAMILY, null,
1741       Permission.Action.READ);
1742     grantOnTable(TEST_UTIL, userQualifier.getShortName(),
1743       TEST_TABLE, TEST_FAMILY, TEST_Q1,
1744       Permission.Action.READ);
1745     try {
1746       AccessTestAction tableRead = new AccessTestAction() {
1747         @Override
1748         public Void run() throws Exception {
1749           checkTablePerms(TEST_UTIL, TEST_TABLE, null, null, Permission.Action.READ);
1750           return null;
1751         }
1752       };
1753 
1754       AccessTestAction columnRead = new AccessTestAction() {
1755         @Override
1756         public Void run() throws Exception {
1757           checkTablePerms(TEST_UTIL, TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ);
1758           return null;
1759         }
1760       };
1761 
1762       AccessTestAction qualifierRead = new AccessTestAction() {
1763         @Override
1764         public Void run() throws Exception {
1765           checkTablePerms(TEST_UTIL, TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ);
1766           return null;
1767         }
1768       };
1769 
1770       AccessTestAction multiQualifierRead = new AccessTestAction() {
1771         @Override
1772         public Void run() throws Exception {
1773           checkTablePerms(TEST_UTIL, TEST_TABLE, new Permission[] {
1774               new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_Q1, Permission.Action.READ),
1775               new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_Q2, Permission.Action.READ), });
1776           return null;
1777         }
1778       };
1779 
1780       AccessTestAction globalAndTableRead = new AccessTestAction() {
1781         @Override
1782         public Void run() throws Exception {
1783           checkTablePerms(TEST_UTIL, TEST_TABLE, new Permission[] {
1784               new Permission(Permission.Action.READ),
1785               new TablePermission(TEST_TABLE, null, (byte[]) null, Permission.Action.READ), });
1786           return null;
1787         }
1788       };
1789 
1790       AccessTestAction noCheck = new AccessTestAction() {
1791         @Override
1792         public Void run() throws Exception {
1793           checkTablePerms(TEST_UTIL, TEST_TABLE, new Permission[0]);
1794           return null;
1795         }
1796       };
1797 
1798       verifyAllowed(tableRead, SUPERUSER, userTable);
1799       verifyDenied(tableRead, userColumn, userQualifier);
1800 
1801       verifyAllowed(columnRead, SUPERUSER, userTable, userColumn);
1802       verifyDenied(columnRead, userQualifier);
1803 
1804       verifyAllowed(qualifierRead, SUPERUSER, userTable, userColumn, userQualifier);
1805 
1806       verifyAllowed(multiQualifierRead, SUPERUSER, userTable, userColumn);
1807       verifyDenied(multiQualifierRead, userQualifier);
1808 
1809       verifyAllowed(globalAndTableRead, SUPERUSER);
1810       verifyDenied(globalAndTableRead, userTable, userColumn, userQualifier);
1811 
1812       verifyAllowed(noCheck, SUPERUSER, userTable, userColumn, userQualifier);
1813 
1814       // --------------------------------------
1815       // test family level multiple permissions
1816       AccessTestAction familyReadWrite = new AccessTestAction() {
1817         @Override
1818         public Void run() throws Exception {
1819           checkTablePerms(TEST_UTIL, TEST_TABLE, TEST_FAMILY, null, Permission.Action.READ,
1820             Permission.Action.WRITE);
1821           return null;
1822         }
1823       };
1824 
1825       verifyAllowed(familyReadWrite, SUPERUSER, USER_OWNER, USER_CREATE, USER_RW);
1826       verifyDenied(familyReadWrite, USER_NONE, USER_RO);
1827 
1828       // --------------------------------------
1829       // check for wrong table region
1830       CheckPermissionsRequest checkRequest =
1831           CheckPermissionsRequest
1832               .newBuilder()
1833               .addPermission(
1834                 AccessControlProtos.Permission
1835                     .newBuilder()
1836                     .setType(AccessControlProtos.Permission.Type.Table)
1837                     .setTablePermission(
1838                       AccessControlProtos.TablePermission.newBuilder()
1839                           .setTableName(ProtobufUtil.toProtoTableName(TEST_TABLE))
1840                           .addAction(AccessControlProtos.Permission.Action.CREATE))).build();
1841       HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
1842       try {
1843         BlockingRpcChannel channel = acl.coprocessorService(new byte[0]);
1844         AccessControlService.BlockingInterface protocol =
1845             AccessControlService.newBlockingStub(channel);
1846         try {
1847           // but ask for TablePermissions for TEST_TABLE
1848           protocol.checkPermissions(null, checkRequest);
1849           fail("this should have thrown CoprocessorException");
1850         } catch (ServiceException ex) {
1851           // expected
1852         }
1853       } finally {
1854         acl.close();
1855       }
1856     } finally {
1857       revokeFromTable(TEST_UTIL, userTable.getShortName(), TEST_TABLE, null, null,
1858         Permission.Action.READ);
1859       revokeFromTable(TEST_UTIL, userColumn.getShortName(), TEST_TABLE, TEST_FAMILY, null,
1860         Permission.Action.READ);
1861       revokeFromTable(TEST_UTIL, userQualifier.getShortName(), TEST_TABLE, TEST_FAMILY, TEST_Q1,
1862         Permission.Action.READ);
1863     }
1864   }
1865 
1866   @Test
1867   public void testStopRegionServer() throws Exception {
1868     AccessTestAction action = new AccessTestAction() {
1869       @Override
1870       public Object run() throws Exception {
1871         ACCESS_CONTROLLER.preStopRegionServer(ObserverContext.createAndPrepare(RSCP_ENV, null));
1872         return null;
1873       }
1874     };
1875 
1876     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
1877     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
1878       USER_GROUP_WRITE, USER_GROUP_CREATE);
1879   }
1880 
1881   @Test
1882   public void testRollWALWriterRequest() throws Exception {
1883     AccessTestAction action = new AccessTestAction() {
1884       @Override
1885       public Object run() throws Exception {
1886         ACCESS_CONTROLLER.preRollWALWriterRequest(ObserverContext.createAndPrepare(RSCP_ENV, null));
1887         return null;
1888       }
1889     };
1890 
1891     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
1892     verifyDenied(action, USER_CREATE, USER_OWNER, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
1893       USER_GROUP_WRITE, USER_GROUP_CREATE);
1894   }
1895 
1896   @Test
1897   public void testOpenRegion() throws Exception {
1898     AccessTestAction action = new AccessTestAction() {
1899       @Override
1900       public Object run() throws Exception {
1901         ACCESS_CONTROLLER.preOpen(ObserverContext.createAndPrepare(RCP_ENV, null));
1902         return null;
1903       }
1904     };
1905 
1906     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
1907     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER, USER_GROUP_CREATE,
1908       USER_GROUP_READ, USER_GROUP_WRITE);
1909   }
1910 
1911   @Test
1912   public void testCloseRegion() throws Exception {
1913     AccessTestAction action = new AccessTestAction() {
1914       @Override
1915       public Object run() throws Exception {
1916         ACCESS_CONTROLLER.preClose(ObserverContext.createAndPrepare(RCP_ENV, null), false);
1917         return null;
1918       }
1919     };
1920 
1921     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
1922     verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER, USER_GROUP_CREATE,
1923       USER_GROUP_READ, USER_GROUP_WRITE);
1924   }
1925 
1926   @Test
1927   public void testSnapshot() throws Exception {
1928     AccessTestAction snapshotAction = new AccessTestAction() {
1929       @Override
1930       public Object run() throws Exception {
1931         ACCESS_CONTROLLER.preSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
1932           null, null);
1933         return null;
1934       }
1935     };
1936 
1937     AccessTestAction deleteAction = new AccessTestAction() {
1938       @Override
1939       public Object run() throws Exception {
1940         ACCESS_CONTROLLER.preDeleteSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
1941           null);
1942         return null;
1943       }
1944     };
1945 
1946     AccessTestAction restoreAction = new AccessTestAction() {
1947       @Override
1948       public Object run() throws Exception {
1949         ACCESS_CONTROLLER.preRestoreSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
1950           null, null);
1951         return null;
1952       }
1953     };
1954 
1955     AccessTestAction cloneAction = new AccessTestAction() {
1956       @Override
1957       public Object run() throws Exception {
1958         ACCESS_CONTROLLER.preCloneSnapshot(ObserverContext.createAndPrepare(CP_ENV, null),
1959           null, null);
1960         return null;
1961       }
1962     };
1963 
1964     verifyAllowed(snapshotAction, SUPERUSER, USER_ADMIN);
1965     verifyDenied(snapshotAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
1966 
1967     verifyAllowed(cloneAction, SUPERUSER, USER_ADMIN);
1968     verifyDenied(deleteAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
1969 
1970     verifyAllowed(restoreAction, SUPERUSER, USER_ADMIN);
1971     verifyDenied(restoreAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
1972 
1973     verifyAllowed(deleteAction, SUPERUSER, USER_ADMIN);
1974     verifyDenied(cloneAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
1975   }
1976 
1977   @Test
1978   public void testGlobalAuthorizationForNewRegisteredRS() throws Exception {
1979     LOG.debug("Test for global authorization for a new registered RegionServer.");
1980     MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
1981 
1982     // Since each RegionServer running on different user, add global
1983     // permissions for the new user.
1984     String currentUser = User.getCurrent().getShortName();
1985     String activeUserForNewRs = currentUser + ".hfs." +
1986       hbaseCluster.getLiveRegionServerThreads().size();
1987     grantGlobal(TEST_UTIL, activeUserForNewRs,
1988       Permission.Action.ADMIN, Permission.Action.CREATE, Permission.Action.READ,
1989         Permission.Action.WRITE);
1990 
1991     final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
1992     HTableDescriptor htd = new HTableDescriptor(TEST_TABLE2);
1993     htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
1994     admin.createTable(htd);
1995     TEST_UTIL.waitUntilAllRegionsAssigned(TEST_TABLE2);
1996 
1997     // Starting a new RegionServer.
1998     JVMClusterUtil.RegionServerThread newRsThread = hbaseCluster
1999         .startRegionServer();
2000     final HRegionServer newRs = newRsThread.getRegionServer();
2001 
2002     // Move region to the new RegionServer.
2003     final HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE2);
2004     try {
2005       NavigableMap<HRegionInfo, ServerName> regions = table
2006           .getRegionLocations();
2007       final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet()
2008           .iterator().next();
2009 
2010       AccessTestAction moveAction = new AccessTestAction() {
2011         @Override
2012         public Object run() throws Exception {
2013           admin.move(firstRegion.getKey().getEncodedNameAsBytes(),
2014               Bytes.toBytes(newRs.getServerName().getServerName()));
2015           return null;
2016         }
2017       };
2018       SUPERUSER.runAs(moveAction);
2019 
2020       final int RETRIES_LIMIT = 10;
2021       int retries = 0;
2022       while (newRs.getOnlineRegions(TEST_TABLE2).size() < 1 && retries < RETRIES_LIMIT) {
2023         LOG.debug("Waiting for region to be opened. Already retried " + retries
2024             + " times.");
2025         try {
2026           Thread.sleep(1000);
2027         } catch (InterruptedException e) {
2028         }
2029         retries++;
2030         if (retries == RETRIES_LIMIT - 1) {
2031           fail("Retry exhaust for waiting region to be opened.");
2032         }
2033       }
2034       // Verify write permission for user "admin2" who has the global
2035       // permissions.
2036       AccessTestAction putAction = new AccessTestAction() {
2037         @Override
2038         public Object run() throws Exception {
2039           Put put = new Put(Bytes.toBytes("test"));
2040           put.add(TEST_FAMILY, Bytes.toBytes("qual"), Bytes.toBytes("value"));
2041           table.put(put);
2042           return null;
2043         }
2044       };
2045       USER_ADMIN.runAs(putAction);
2046     } finally {
2047       table.close();
2048     }
2049   }
2050 
2051   @Test
2052   public void testTableDescriptorsEnumeration() throws Exception {
2053     User TABLE_ADMIN = User.createUserForTesting(conf, "UserA", new String[0]);
2054 
2055     // Grant TABLE ADMIN privs
2056     grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(),
2057       TEST_TABLE, null, null,
2058       Permission.Action.ADMIN);
2059     try {
2060       AccessTestAction listTablesAction = new AccessTestAction() {
2061         @Override
2062         public Object run() throws Exception {
2063           HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
2064           try {
2065             admin.listTables();
2066           } finally {
2067             admin.close();
2068           }
2069           return null;
2070         }
2071       };
2072 
2073       AccessTestAction getTableDescAction = new AccessTestAction() {
2074         @Override
2075         public Object run() throws Exception {
2076           HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
2077           try {
2078             admin.getTableDescriptor(TEST_TABLE);
2079           } finally {
2080             admin.close();
2081           }
2082           return null;
2083         }
2084       };
2085 
2086       verifyAllowed(listTablesAction, SUPERUSER, USER_ADMIN, USER_GROUP_ADMIN);
2087       verifyDenied(listTablesAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, TABLE_ADMIN,
2088         USER_GROUP_READ, USER_GROUP_WRITE, USER_GROUP_CREATE);
2089 
2090       verifyAllowed(getTableDescAction, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER,
2091         TABLE_ADMIN, USER_GROUP_ADMIN, USER_GROUP_CREATE);
2092       verifyDenied(getTableDescAction, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
2093         USER_GROUP_WRITE);
2094     } finally {
2095       // Cleanup, revoke TABLE ADMIN privs
2096       revokeFromTable(TEST_UTIL, TABLE_ADMIN.getShortName(), TEST_TABLE, null, null,
2097         Permission.Action.ADMIN);
2098     }
2099   }
2100 
2101   @Test
2102   public void testTableDeletion() throws Exception {
2103     User TABLE_ADMIN = User.createUserForTesting(conf, "TestUser", new String[0]);
2104     final TableName tname = TableName.valueOf("testTableDeletion");
2105     createTestTable(tname);
2106 
2107     // Grant TABLE ADMIN privs
2108     grantOnTable(TEST_UTIL, TABLE_ADMIN.getShortName(), tname, null, null, Permission.Action.ADMIN);
2109 
2110     AccessTestAction deleteTableAction = new AccessTestAction() {
2111       @Override
2112       public Object run() throws Exception {
2113         HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
2114         try {
2115           admin.disableTable(tname);
2116           admin.deleteTable(tname);
2117         } finally {
2118           admin.close();
2119         }
2120         return null;
2121       }
2122     };
2123 
2124     verifyDenied(deleteTableAction, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ,
2125       USER_GROUP_WRITE);
2126     verifyAllowed(deleteTableAction, TABLE_ADMIN);
2127   }
2128 
2129   @Test
2130   public void testNamespaceUserGrant() throws Exception {
2131     AccessTestAction getAction = new AccessTestAction() {
2132       @Override
2133       public Object run() throws Exception {
2134         HTable t = new HTable(conf, TEST_TABLE);
2135         try {
2136           return t.get(new Get(TEST_ROW));
2137         } finally {
2138           t.close();
2139         }
2140       }
2141     };
2142 
2143     verifyDenied(getAction, USER_NONE);
2144 
2145     // Grant namespace READ to USER_NONE, this should supersede any table permissions
2146     grantOnNamespace(TEST_UTIL, USER_NONE.getShortName(),
2147       TEST_TABLE.getNamespaceAsString(),
2148       Permission.Action.READ);
2149 
2150     // Now USER_NONE should be able to read also
2151     verifyAllowed(getAction, USER_NONE);
2152   }
2153 
2154   @Test
2155   public void testAccessControlClientGrantRevoke() throws Exception {
2156     // Create user for testing, who has no READ privileges by default.
2157     User testGrantRevoke = User.createUserForTesting(conf, "testGrantRevoke", new String[0]);
2158     AccessTestAction getAction = new AccessTestAction() {
2159       @Override
2160       public Object run() throws Exception {
2161         HTable t = new HTable(conf, TEST_TABLE);
2162         try {
2163           return t.get(new Get(TEST_ROW));
2164         } finally {
2165           t.close();
2166         }
2167       }
2168     };
2169 
2170     verifyDenied(getAction, testGrantRevoke);
2171 
2172     // Grant table READ permissions to testGrantRevoke.
2173     try {
2174       grantOnTableUsingAccessControlClient(TEST_UTIL, conf, testGrantRevoke.getShortName(),
2175           TEST_TABLE, null, null, Permission.Action.READ);
2176     } catch (Throwable e) {
2177       LOG.error("error during call of AccessControlClient.grant. " + e.getStackTrace());
2178     }
2179 
2180     // Now testGrantRevoke should be able to read also
2181     verifyAllowed(getAction, testGrantRevoke);
2182 
2183     // Revoke table READ permission to testGrantRevoke.
2184     try {
2185       revokeFromTableUsingAccessControlClient(TEST_UTIL, conf, testGrantRevoke.getShortName(),
2186           TEST_TABLE, null, null, Permission.Action.READ);
2187     } catch (Throwable e) {
2188       LOG.error("error during call of AccessControlClient.revoke " + e.getStackTrace());
2189     }
2190 
2191     // Now testGrantRevoke shouldn't be able read
2192     verifyDenied(getAction, testGrantRevoke);
2193   }
2194 
2195   @Test
2196   public void testAccessControlClientGlobalGrantRevoke() throws Exception {
2197     // Create user for testing, who has no READ privileges by default.
2198     User testGlobalGrantRevoke = User.createUserForTesting(conf,
2199       "testGlobalGrantRevoke", new String[0]);
2200     AccessTestAction getAction = new AccessTestAction() {
2201       @Override
2202       public Object run() throws Exception {
2203         HTable t = new HTable(conf, TEST_TABLE);
2204         try {
2205           return t.get(new Get(TEST_ROW));
2206         } finally {
2207           t.close();
2208         }
2209       }
2210     };
2211 
2212     verifyDenied(getAction, testGlobalGrantRevoke);
2213 
2214     // Grant table READ permissions to testGlobalGrantRevoke.
2215     String userName = testGlobalGrantRevoke.getShortName();
2216     try {
2217       grantGlobalUsingAccessControlClient(TEST_UTIL, conf, userName,
2218         Permission.Action.READ);
2219     } catch (Throwable e) {
2220       LOG.error("error during call of AccessControlClient.grant. ", e);
2221     }
2222     try {
2223       // Now testGlobalGrantRevoke should be able to read also
2224       verifyAllowed(getAction, testGlobalGrantRevoke);
2225 
2226       // Revoke table READ permission to testGlobalGrantRevoke.
2227       try {
2228         revokeGlobalUsingAccessControlClient(TEST_UTIL, conf, userName, Permission.Action.READ);
2229       } catch (Throwable e) {
2230         LOG.error("error during call of AccessControlClient.revoke ", e);
2231       }
2232 
2233       // Now testGlobalGrantRevoke shouldn't be able read
2234       verifyDenied(getAction, testGlobalGrantRevoke);
2235     } finally {
2236       revokeGlobal(TEST_UTIL, userName, Permission.Action.READ);
2237     }
2238   }
2239 
2240   @Test
2241   public void testAccessControlClientGrantRevokeOnNamespace() throws Exception {
2242     // Create user for testing, who has no READ privileges by default.
2243     User testNS = User.createUserForTesting(conf, "testNS", new String[0]);
2244     AccessTestAction getAction = new AccessTestAction() {
2245       @Override
2246       public Object run() throws Exception {
2247         HTable t = new HTable(conf, TEST_TABLE);
2248         try {
2249           return t.get(new Get(TEST_ROW));
2250         } finally {
2251           t.close();
2252         }
2253       }
2254     };
2255 
2256     verifyDenied(getAction, testNS);
2257 
2258     // Grant namespace READ to testNS, this should supersede any table permissions
2259     String userName = testNS.getShortName();
2260     String namespace = TEST_TABLE.getNamespaceAsString();
2261     try {
2262       grantOnNamespaceUsingAccessControlClient(TEST_UTIL, conf, userName,
2263           namespace, Permission.Action.READ);
2264     } catch (Throwable e) {
2265       LOG.error("error during call of AccessControlClient.grant. " + e.getStackTrace());
2266     }
2267 
2268     try {
2269       // Now testNS should be able to read also
2270       verifyAllowed(getAction, testNS);
2271 
2272       // Revoke namespace READ to testNS, this should supersede any table permissions
2273       try {
2274         revokeFromNamespaceUsingAccessControlClient(TEST_UTIL, conf, userName, namespace,
2275           Permission.Action.READ);
2276       } catch (Throwable e) {
2277         LOG.error("error during call of AccessControlClient.revoke " + e.getStackTrace());
2278       }
2279 
2280       // Now testNS shouldn't be able read
2281       verifyDenied(getAction, testNS);
2282     } finally {
2283       revokeFromNamespace(TEST_UTIL, userName, namespace, Permission.Action.READ);
2284     }
2285   }
2286 
2287 
2288   public static class PingCoprocessor extends PingService implements Coprocessor,
2289       CoprocessorService {
2290 
2291     @Override
2292     public void start(CoprocessorEnvironment env) throws IOException { }
2293 
2294     @Override
2295     public void stop(CoprocessorEnvironment env) throws IOException { }
2296 
2297     @Override
2298     public Service getService() {
2299       return this;
2300     }
2301 
2302     @Override
2303     public void ping(RpcController controller, PingRequest request,
2304         RpcCallback<PingResponse> callback) {
2305       callback.run(PingResponse.newBuilder().setPong("Pong!").build());
2306     }
2307 
2308     @Override
2309     public void count(RpcController controller, CountRequest request,
2310         RpcCallback<CountResponse> callback) {
2311       callback.run(CountResponse.newBuilder().build());
2312     }
2313 
2314     @Override
2315     public void increment(RpcController controller, IncrementCountRequest requet,
2316         RpcCallback<IncrementCountResponse> callback) {
2317       callback.run(IncrementCountResponse.newBuilder().build());
2318     }
2319 
2320     @Override
2321     public void hello(RpcController controller, HelloRequest request,
2322         RpcCallback<HelloResponse> callback) {
2323       callback.run(HelloResponse.newBuilder().setResponse("Hello!").build());
2324     }
2325 
2326     @Override
2327     public void noop(RpcController controller, NoopRequest request,
2328         RpcCallback<NoopResponse> callback) {
2329       callback.run(NoopResponse.newBuilder().build());
2330     }
2331   }
2332 
2333   @Test
2334   public void testCoprocessorExec() throws Exception {
2335     // Set up our ping endpoint service on all regions of our test table
2336     for (JVMClusterUtil.RegionServerThread thread:
2337         TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads()) {
2338       HRegionServer rs = thread.getRegionServer();
2339       for (HRegion region: rs.getOnlineRegions(TEST_TABLE)) {
2340         region.getCoprocessorHost().load(PingCoprocessor.class,
2341           Coprocessor.PRIORITY_USER, conf);
2342       }
2343     }
2344 
2345     // Create users for testing, and grant EXEC privileges on our test table
2346     // only to user A
2347     User userA = User.createUserForTesting(conf, "UserA", new String[0]);
2348     User userB = User.createUserForTesting(conf, "UserB", new String[0]);
2349 
2350     grantOnTable(TEST_UTIL, userA.getShortName(),
2351       TEST_TABLE, null, null,
2352       Permission.Action.EXEC);
2353     try {
2354       // Create an action for invoking our test endpoint
2355       AccessTestAction execEndpointAction = new AccessTestAction() {
2356         @Override
2357         public Object run() throws Exception {
2358           HTable t = new HTable(conf, TEST_TABLE);
2359           try {
2360             BlockingRpcChannel service = t.coprocessorService(HConstants.EMPTY_BYTE_ARRAY);
2361             PingCoprocessor.newBlockingStub(service).noop(null, NoopRequest.newBuilder().build());
2362           } finally {
2363             t.close();
2364           }
2365           return null;
2366         }
2367       };
2368 
2369       // Verify that EXEC permission is checked correctly
2370       verifyDenied(execEndpointAction, userB);
2371       verifyAllowed(execEndpointAction, userA);
2372 
2373       // Now grant EXEC to the entire namespace to user B
2374       grantOnNamespace(TEST_UTIL, userB.getShortName(), TEST_TABLE.getNamespaceAsString(),
2375         Permission.Action.EXEC);
2376 
2377       // User B should now be allowed also
2378       verifyAllowed(execEndpointAction, userA, userB);
2379     } finally {
2380       // Cleanup, revoke the userA privileges
2381       revokeFromTable(TEST_UTIL, userA.getShortName(), TEST_TABLE, null, null,
2382         Permission.Action.EXEC);
2383     }
2384   }
2385 
2386   @Test
2387   public void testReservedCellTags() throws Exception {
2388     AccessTestAction putWithReservedTag = new AccessTestAction() {
2389       @Override
2390       public Object run() throws Exception {
2391         HTable t = new HTable(conf, TEST_TABLE);
2392         try {
2393           KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
2394             HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY,
2395             new Tag[] { new Tag(AccessControlLists.ACL_TAG_TYPE,
2396               ProtobufUtil.toUsersAndPermissions(USER_OWNER.getShortName(),
2397                 new Permission(Permission.Action.READ)).toByteArray()) });
2398           t.put(new Put(TEST_ROW).add(kv));
2399         } finally {
2400           t.close();
2401         }
2402         return null;
2403       }
2404     };
2405 
2406     // Current user is superuser
2407     verifyAllowed(putWithReservedTag, User.getCurrent());
2408     // No other user should be allowed
2409     verifyDenied(putWithReservedTag, USER_OWNER, USER_ADMIN, USER_CREATE, USER_RW, USER_RO);
2410   }
2411 
2412   @Test
2413   public void testGetNamespacePermission() throws Exception {
2414     String namespace = "testNamespace";
2415     NamespaceDescriptor desc = NamespaceDescriptor.create(namespace).build();
2416     createNamespace(TEST_UTIL, desc);
2417     grantOnNamespace(TEST_UTIL, USER_NONE.getShortName(), namespace, Permission.Action.READ);
2418     try {
2419       List<UserPermission> namespacePermissions = AccessControlClient.getUserPermissions(conf,
2420       AccessControlLists.toNamespaceEntry(namespace));
2421       assertTrue(namespacePermissions != null);
2422       assertTrue(namespacePermissions.size() == 1);
2423     } catch (Throwable thw) {
2424       throw new HBaseException(thw);
2425     }
2426     deleteNamespace(TEST_UTIL, namespace);
2427   }
2428 
2429   @Test
2430   public void testTruncatePerms() throws Exception {
2431     try {
2432       List<UserPermission> existingPerms =
2433           AccessControlClient.getUserPermissions(conf, TEST_TABLE.getNameAsString());
2434       assertTrue(existingPerms != null);
2435       assertTrue(existingPerms.size() > 1);
2436       TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE);
2437       TEST_UTIL.getHBaseAdmin().truncateTable(TEST_TABLE, true);
2438       List<UserPermission> perms =
2439           AccessControlClient.getUserPermissions(conf, TEST_TABLE.getNameAsString());
2440       assertTrue(perms != null);
2441       assertEquals(existingPerms.size(), perms.size());
2442     } catch (Throwable e) {
2443       throw new HBaseIOException(e);
2444     }
2445   }
2446 
2447   private void verifyAnyCreate(AccessTestAction action) throws Exception {
2448     verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_OWNER, USER_CREATE, USER_ADMIN_CF,
2449       USER_GROUP_CREATE);
2450     verifyDenied(action, USER_NONE, USER_RO, USER_RW, USER_GROUP_READ, USER_GROUP_WRITE,
2451       USER_GROUP_ADMIN);
2452   }
2453 
2454   @Test
2455   public void testPrepareAndCleanBulkLoad() throws Exception {
2456     AccessTestAction prepareBulkLoadAction = new AccessTestAction() {
2457       @Override
2458       public Object run() throws Exception {
2459         ACCESS_CONTROLLER.prePrepareBulkLoad(ObserverContext.createAndPrepare(RCP_ENV, null), null);
2460         return null;
2461       }
2462     };
2463     AccessTestAction cleanupBulkLoadAction = new AccessTestAction() {
2464       @Override
2465       public Object run() throws Exception {
2466         ACCESS_CONTROLLER.preCleanupBulkLoad(ObserverContext.createAndPrepare(RCP_ENV, null), null);
2467         return null;
2468       }
2469     };
2470     verifyAnyCreate(prepareBulkLoadAction);
2471     verifyAnyCreate(cleanupBulkLoadAction);
2472   }
2473 
2474   @Test
2475   public void testReplicateLogEntries() throws Exception {
2476     AccessTestAction replicateLogEntriesAction = new AccessTestAction() {
2477       @Override
2478       public Object run() throws Exception {
2479         ACCESS_CONTROLLER.preReplicateLogEntries(ObserverContext.createAndPrepare(RSCP_ENV, null),
2480           null, null);
2481         ACCESS_CONTROLLER.postReplicateLogEntries(ObserverContext.createAndPrepare(RSCP_ENV, null),
2482           null, null);
2483         return null;
2484       }
2485     };
2486 
2487     verifyAllowed(replicateLogEntriesAction, SUPERUSER, USER_ADMIN, USER_GROUP_WRITE);
2488     verifyDenied(replicateLogEntriesAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER,
2489       USER_GROUP_READ, USER_GROUP_ADMIN, USER_GROUP_CREATE);
2490   }
2491 }