001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.security.access;
019
020import static org.junit.Assert.assertEquals;
021
022import java.util.HashMap;
023import java.util.List;
024import java.util.Map;
025import org.apache.hadoop.conf.Configuration;
026import org.apache.hadoop.hbase.AuthUtil;
027import org.apache.hadoop.hbase.Cell;
028import org.apache.hadoop.hbase.Coprocessor;
029import org.apache.hadoop.hbase.HBaseClassTestRule;
030import org.apache.hadoop.hbase.HBaseTestingUtil;
031import org.apache.hadoop.hbase.HConstants;
032import org.apache.hadoop.hbase.TableNameTestRule;
033import org.apache.hadoop.hbase.TableNotFoundException;
034import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
035import org.apache.hadoop.hbase.client.Connection;
036import org.apache.hadoop.hbase.client.ConnectionFactory;
037import org.apache.hadoop.hbase.client.Delete;
038import org.apache.hadoop.hbase.client.Get;
039import org.apache.hadoop.hbase.client.Increment;
040import org.apache.hadoop.hbase.client.Put;
041import org.apache.hadoop.hbase.client.Result;
042import org.apache.hadoop.hbase.client.ResultScanner;
043import org.apache.hadoop.hbase.client.Scan;
044import org.apache.hadoop.hbase.client.Table;
045import org.apache.hadoop.hbase.client.TableDescriptor;
046import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
047import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
048import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
049import org.apache.hadoop.hbase.security.User;
050import org.apache.hadoop.hbase.security.access.Permission.Action;
051import org.apache.hadoop.hbase.testclassification.MediumTests;
052import org.apache.hadoop.hbase.testclassification.SecurityTests;
053import org.apache.hadoop.hbase.util.Bytes;
054import org.apache.hadoop.hbase.util.Threads;
055import org.junit.After;
056import org.junit.AfterClass;
057import org.junit.Before;
058import org.junit.BeforeClass;
059import org.junit.ClassRule;
060import org.junit.Rule;
061import org.junit.Test;
062import org.junit.experimental.categories.Category;
063import org.slf4j.Logger;
064import org.slf4j.LoggerFactory;
065
066import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
067
068@Category({ SecurityTests.class, MediumTests.class })
069public class TestCellACLs extends SecureTestUtil {
070
071  @ClassRule
072  public static final HBaseClassTestRule CLASS_RULE =
073    HBaseClassTestRule.forClass(TestCellACLs.class);
074
075  private static final Logger LOG = LoggerFactory.getLogger(TestCellACLs.class);
076
077  @Rule
078  public TableNameTestRule testTable = new TableNameTestRule();
079  private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
080  private static final byte[] TEST_FAMILY = Bytes.toBytes("f1");
081  private static final byte[] TEST_ROW = Bytes.toBytes("cellpermtest");
082  private static final byte[] TEST_Q1 = Bytes.toBytes("q1");
083  private static final byte[] TEST_Q2 = Bytes.toBytes("q2");
084  private static final byte[] TEST_Q3 = Bytes.toBytes("q3");
085  private static final byte[] TEST_Q4 = Bytes.toBytes("q4");
086  private static final byte[] ZERO = Bytes.toBytes(0L);
087  private static final byte[] ONE = Bytes.toBytes(1L);
088
089  private static Configuration conf;
090
091  private static final String GROUP = "group";
092  private static User GROUP_USER;
093  private static User USER_OWNER;
094  private static User USER_OTHER;
095  private static String[] usersAndGroups;
096
097  @BeforeClass
098  public static void setupBeforeClass() throws Exception {
099    // setup configuration
100    conf = TEST_UTIL.getConfiguration();
101    conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
102    // Enable security
103    enableSecurity(conf);
104    // Verify enableSecurity sets up what we require
105    verifyConfiguration(conf);
106
107    // We expect 0.98 cell ACL semantics
108    conf.setBoolean(AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT, false);
109
110    TEST_UTIL.startMiniCluster();
111    MasterCoprocessorHost cpHost =
112      TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
113    cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
114    AccessController ac = cpHost.findCoprocessor(AccessController.class);
115    cpHost.createEnvironment(ac, Coprocessor.PRIORITY_HIGHEST, 1, conf);
116    RegionServerCoprocessorHost rsHost =
117      TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).getRegionServerCoprocessorHost();
118    rsHost.createEnvironment(ac, Coprocessor.PRIORITY_HIGHEST, 1, conf);
119
120    // Wait for the ACL table to become available
121    TEST_UTIL.waitTableEnabled(PermissionStorage.ACL_TABLE_NAME);
122
123    // create a set of test users
124    USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
125    USER_OTHER = User.createUserForTesting(conf, "other", new String[0]);
126    GROUP_USER = User.createUserForTesting(conf, "group_user", new String[] { GROUP });
127
128    usersAndGroups = new String[] { USER_OTHER.getShortName(), AuthUtil.toGroupEntry(GROUP) };
129
130    // Grant table creation permission to USER_OWNER
131    grantGlobal(TEST_UTIL, USER_OWNER.getShortName(), Action.CREATE);
132  }
133
134  @AfterClass
135  public static void tearDownAfterClass() throws Exception {
136    TEST_UTIL.shutdownMiniCluster();
137  }
138
139  @Before
140  public void setUp() throws Exception {
141    // Create the test table (owner added to the _acl_ table)
142    TableDescriptor tableDescriptor =
143      TableDescriptorBuilder.newBuilder(testTable.getTableName()).setColumnFamily(
144        ColumnFamilyDescriptorBuilder.newBuilder(TEST_FAMILY).setMaxVersions(4).build()).build();
145    createTable(TEST_UTIL, USER_OWNER, tableDescriptor, new byte[][] { Bytes.toBytes("s") });
146    TEST_UTIL.waitTableEnabled(testTable.getTableName());
147    LOG.info("Sleeping a second because of HBASE-12581");
148    Threads.sleep(1000);
149  }
150
151  @Test
152  public void testCellPermissions() throws Exception {
153    // store two sets of values, one store with a cell level ACL, and one without
154    verifyAllowed(new AccessTestAction() {
155      @Override
156      public Object run() throws Exception {
157        try (Connection connection = ConnectionFactory.createConnection(conf);
158          Table t = connection.getTable(testTable.getTableName())) {
159          Put p;
160          // with ro ACL
161          p = new Put(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, ZERO);
162          p.setACL(prepareCellPermissions(usersAndGroups, Action.READ));
163          t.put(p);
164          // with rw ACL
165          p = new Put(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, ZERO);
166          p.setACL(prepareCellPermissions(usersAndGroups, Action.READ, Action.WRITE));
167          t.put(p);
168          // no ACL
169          p = new Put(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3, ZERO).addColumn(TEST_FAMILY,
170            TEST_Q4, ZERO);
171          t.put(p);
172        }
173        return null;
174      }
175    }, USER_OWNER);
176
177    /* ---- Gets ---- */
178
179    AccessTestAction getQ1 = new AccessTestAction() {
180      @Override
181      public Object run() throws Exception {
182        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1);
183        try (Connection connection = ConnectionFactory.createConnection(conf);
184          Table t = connection.getTable(testTable.getTableName())) {
185          return t.get(get).listCells();
186        }
187      }
188    };
189
190    AccessTestAction getQ2 = new AccessTestAction() {
191      @Override
192      public Object run() throws Exception {
193        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2);
194        try (Connection connection = ConnectionFactory.createConnection(conf);
195          Table t = connection.getTable(testTable.getTableName())) {
196          return t.get(get).listCells();
197        }
198      }
199    };
200
201    AccessTestAction getQ3 = new AccessTestAction() {
202      @Override
203      public Object run() throws Exception {
204        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3);
205        try (Connection connection = ConnectionFactory.createConnection(conf);
206          Table t = connection.getTable(testTable.getTableName())) {
207          return t.get(get).listCells();
208        }
209      }
210    };
211
212    AccessTestAction getQ4 = new AccessTestAction() {
213      @Override
214      public Object run() throws Exception {
215        Get get = new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q4);
216        try (Connection connection = ConnectionFactory.createConnection(conf);
217          Table t = connection.getTable(testTable.getTableName())) {
218          return t.get(get).listCells();
219        }
220      }
221    };
222
223    // Confirm special read access set at cell level
224
225    verifyAllowed(getQ1, USER_OTHER, GROUP_USER);
226    verifyAllowed(getQ2, USER_OTHER, GROUP_USER);
227
228    // Confirm this access does not extend to other cells
229
230    verifyIfNull(getQ3, USER_OTHER, GROUP_USER);
231    verifyIfNull(getQ4, USER_OTHER, GROUP_USER);
232
233    /* ---- Scans ---- */
234
235    // check that a scan over the test data returns the expected number of KVs
236
237    final List<Cell> scanResults = Lists.newArrayList();
238
239    AccessTestAction scanAction = new AccessTestAction() {
240      @Override
241      public List<Cell> run() throws Exception {
242        Scan scan = new Scan();
243        scan.withStartRow(TEST_ROW);
244        scan.withStopRow(Bytes.add(TEST_ROW, new byte[] { 0 }));
245        scan.addFamily(TEST_FAMILY);
246        Connection connection = ConnectionFactory.createConnection(conf);
247        Table t = connection.getTable(testTable.getTableName());
248        try {
249          ResultScanner scanner = t.getScanner(scan);
250          Result result = null;
251          do {
252            result = scanner.next();
253            if (result != null) {
254              scanResults.addAll(result.listCells());
255            }
256          } while (result != null);
257        } finally {
258          t.close();
259          connection.close();
260        }
261        return scanResults;
262      }
263    };
264
265    // owner will see all values
266    scanResults.clear();
267    verifyAllowed(scanAction, USER_OWNER);
268    assertEquals(4, scanResults.size());
269
270    // other user will see 2 values
271    scanResults.clear();
272    verifyAllowed(scanAction, USER_OTHER);
273    assertEquals(2, scanResults.size());
274
275    scanResults.clear();
276    verifyAllowed(scanAction, GROUP_USER);
277    assertEquals(2, scanResults.size());
278
279    /* ---- Increments ---- */
280
281    AccessTestAction incrementQ1 = new AccessTestAction() {
282      @Override
283      public Object run() throws Exception {
284        Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, 1L);
285        try (Connection connection = ConnectionFactory.createConnection(conf);
286          Table t = connection.getTable(testTable.getTableName())) {
287          t.increment(i);
288        }
289        return null;
290      }
291    };
292
293    AccessTestAction incrementQ2 = new AccessTestAction() {
294      @Override
295      public Object run() throws Exception {
296        Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L);
297        try (Connection connection = ConnectionFactory.createConnection(conf);
298          Table t = connection.getTable(testTable.getTableName())) {
299          t.increment(i);
300        }
301        return null;
302      }
303    };
304
305    AccessTestAction incrementQ2newDenyACL = new AccessTestAction() {
306      @Override
307      public Object run() throws Exception {
308        Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q2, 1L);
309        // Tag this increment with an ACL that denies write permissions to USER_OTHER and GROUP
310        i.setACL(prepareCellPermissions(usersAndGroups, Action.READ));
311        try (Connection connection = ConnectionFactory.createConnection(conf);
312          Table t = connection.getTable(testTable.getTableName())) {
313          t.increment(i);
314        }
315        return null;
316      }
317    };
318
319    AccessTestAction incrementQ3 = new AccessTestAction() {
320      @Override
321      public Object run() throws Exception {
322        Increment i = new Increment(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q3, 1L);
323        try (Connection connection = ConnectionFactory.createConnection(conf);
324          Table t = connection.getTable(testTable.getTableName())) {
325          t.increment(i);
326        }
327        return null;
328      }
329    };
330
331    verifyDenied(incrementQ1, USER_OTHER, GROUP_USER);
332    verifyDenied(incrementQ3, USER_OTHER, GROUP_USER);
333
334    // We should be able to increment until the permissions are revoked (including the action in
335    // which permissions are revoked, the previous ACL will be carried forward)
336    verifyAllowed(incrementQ2, USER_OTHER, GROUP_USER);
337    verifyAllowed(incrementQ2newDenyACL, USER_OTHER);
338    // But not again after we denied ourselves write permission with an ACL
339    // update
340    verifyDenied(incrementQ2, USER_OTHER, GROUP_USER);
341
342    /* ---- Deletes ---- */
343
344    AccessTestAction deleteFamily = new AccessTestAction() {
345      @Override
346      public Object run() throws Exception {
347        Delete delete = new Delete(TEST_ROW).addFamily(TEST_FAMILY);
348        try (Connection connection = ConnectionFactory.createConnection(conf);
349          Table t = connection.getTable(testTable.getTableName())) {
350          t.delete(delete);
351        }
352        return null;
353      }
354    };
355
356    AccessTestAction deleteQ1 = new AccessTestAction() {
357      @Override
358      public Object run() throws Exception {
359        Delete delete = new Delete(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1);
360        try (Connection connection = ConnectionFactory.createConnection(conf);
361          Table t = connection.getTable(testTable.getTableName())) {
362          t.delete(delete);
363        }
364        return null;
365      }
366    };
367
368    verifyDenied(deleteFamily, USER_OTHER, GROUP_USER);
369    verifyDenied(deleteQ1, USER_OTHER, GROUP_USER);
370    verifyAllowed(deleteQ1, USER_OWNER);
371  }
372
373  /**
374   * Insure we are not granting access in the absence of any cells found when scanning for covered
375   * cells.
376   */
377  @Test
378  public void testCoveringCheck() throws Exception {
379    // Grant read access to USER_OTHER
380    grantOnTable(TEST_UTIL, USER_OTHER.getShortName(), testTable.getTableName(), TEST_FAMILY, null,
381      Action.READ);
382    // Grant read access to GROUP
383    grantOnTable(TEST_UTIL, AuthUtil.toGroupEntry(GROUP), testTable.getTableName(), TEST_FAMILY,
384      null, Action.READ);
385
386    // A write by USER_OTHER should be denied.
387    // This is where we could have a big problem if there is an error in the
388    // covering check logic.
389    verifyUserDeniedForWrite(USER_OTHER, ZERO);
390    // A write by GROUP_USER from group GROUP should be denied.
391    verifyUserDeniedForWrite(GROUP_USER, ZERO);
392
393    // Add the cell
394    verifyAllowed(new AccessTestAction() {
395      @Override
396      public Object run() throws Exception {
397        try (Connection connection = ConnectionFactory.createConnection(conf);
398          Table t = connection.getTable(testTable.getTableName())) {
399          Put p;
400          p = new Put(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, ZERO);
401          t.put(p);
402        }
403        return null;
404      }
405    }, USER_OWNER);
406
407    // A write by USER_OTHER should still be denied, just to make sure
408    verifyUserDeniedForWrite(USER_OTHER, ONE);
409    // A write by GROUP_USER from group GROUP should still be denied
410    verifyUserDeniedForWrite(GROUP_USER, ONE);
411
412    // A read by USER_OTHER should be allowed, just to make sure
413    verifyUserAllowedForRead(USER_OTHER);
414    // A read by GROUP_USER from group GROUP should be allowed
415    verifyUserAllowedForRead(GROUP_USER);
416  }
417
418  private void verifyUserDeniedForWrite(final User user, final byte[] value) throws Exception {
419    verifyDenied(new AccessTestAction() {
420      @Override
421      public Object run() throws Exception {
422        try (Connection connection = ConnectionFactory.createConnection(conf);
423          Table t = connection.getTable(testTable.getTableName())) {
424          Put p;
425          p = new Put(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1, value);
426          t.put(p);
427        }
428        return null;
429      }
430    }, user);
431  }
432
433  private void verifyUserAllowedForRead(final User user) throws Exception {
434    verifyAllowed(new AccessTestAction() {
435      @Override
436      public Object run() throws Exception {
437        try (Connection connection = ConnectionFactory.createConnection(conf);
438          Table t = connection.getTable(testTable.getTableName())) {
439          return t.get(new Get(TEST_ROW).addColumn(TEST_FAMILY, TEST_Q1));
440        }
441      }
442    }, user);
443  }
444
445  private Map<String, Permission> prepareCellPermissions(String[] users, Action... action) {
446    Map<String, Permission> perms = new HashMap<>(2);
447    for (String user : users) {
448      perms.put(user, new Permission(action));
449    }
450    return perms;
451  }
452
453  @After
454  public void tearDown() throws Exception {
455    // Clean the _acl_ table
456    try {
457      TEST_UTIL.deleteTable(testTable.getTableName());
458    } catch (TableNotFoundException ex) {
459      // Test deleted the table, no problem
460      LOG.info("Test deleted table " + testTable.getTableName());
461    }
462    assertEquals(0, PermissionStorage.getTablePermissions(conf, testTable.getTableName()).size());
463  }
464}