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 java.io.IOException;
021import java.util.ArrayList;
022import java.util.List;
023import java.util.regex.Pattern;
024import org.apache.commons.lang3.StringUtils;
025import org.apache.hadoop.hbase.HConstants;
026import org.apache.hadoop.hbase.MasterNotRunningException;
027import org.apache.hadoop.hbase.NamespaceDescriptor;
028import org.apache.hadoop.hbase.TableName;
029import org.apache.hadoop.hbase.ZooKeeperConnectionException;
030import org.apache.hadoop.hbase.client.Admin;
031import org.apache.hadoop.hbase.client.Connection;
032import org.apache.hadoop.hbase.client.TableDescriptor;
033import org.apache.hadoop.hbase.client.security.SecurityCapability;
034import org.apache.hadoop.hbase.util.Bytes;
035import org.apache.yetus.audience.InterfaceAudience;
036
037/**
038 * Utility client for doing access control admin operations.
039 */
040@InterfaceAudience.Public
041public class AccessControlClient {
042  public static final TableName ACL_TABLE_NAME =
043    TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl");
044
045  /**
046   * Return true if authorization is supported and enabled
047   * @param connection The connection to use
048   * @return true if authorization is supported and enabled, false otherwise
049   */
050  public static boolean isAuthorizationEnabled(Connection connection) throws IOException {
051    return connection.getAdmin().getSecurityCapabilities()
052      .contains(SecurityCapability.AUTHORIZATION);
053  }
054
055  /**
056   * Return true if cell authorization is supported and enabled
057   * @param connection The connection to use
058   * @return true if cell authorization is supported and enabled, false otherwise
059   */
060  public static boolean isCellAuthorizationEnabled(Connection connection) throws IOException {
061    return connection.getAdmin().getSecurityCapabilities()
062      .contains(SecurityCapability.CELL_AUTHORIZATION);
063  }
064
065  /**
066   * Grants permission on the specified table for the specified user
067   * @param connection               The Connection instance to use
068   * @param tableName                the table name
069   * @param userName                 the user name
070   * @param family                   the column family
071   * @param qual                     the column qualifier
072   * @param mergeExistingPermissions If set to false, later granted permissions will override
073   *                                 previous granted permissions. otherwise, it'll merge with
074   *                                 previous granted permissions.
075   * @param actions                  the actions
076   */
077  private static void grant(Connection connection, final TableName tableName, final String userName,
078    final byte[] family, final byte[] qual, boolean mergeExistingPermissions,
079    final Permission.Action... actions) throws Throwable {
080    connection.getAdmin().grant(new UserPermission(userName, Permission.newBuilder(tableName)
081      .withFamily(family).withQualifier(qual).withActions(actions).build()),
082      mergeExistingPermissions);
083  }
084
085  /**
086   * Grants permission on the specified table for the specified user. If permissions for a specified
087   * user exists, later granted permissions will override previous granted permissions.
088   * @param connection The Connection instance to use
089   * @param tableName  the table name
090   * @param userName   the user name
091   * @param family     the column family
092   * @param qual       the column qualifier
093   * @param actions    the actions
094   */
095  public static void grant(Connection connection, final TableName tableName, final String userName,
096    final byte[] family, final byte[] qual, final Permission.Action... actions) throws Throwable {
097    grant(connection, tableName, userName, family, qual, true, actions);
098  }
099
100  /**
101   * Grants permission on the specified namespace for the specified user.
102   * @param connection               The Connection instance to use
103   * @param namespace                the namespace
104   * @param userName                 the user name
105   * @param mergeExistingPermissions If set to false, later granted permissions will override
106   *                                 previous granted permissions. otherwise, it'll merge with
107   *                                 previous granted permissions.
108   * @param actions                  the actions
109   */
110  private static void grant(Connection connection, final String namespace, final String userName,
111    boolean mergeExistingPermissions, final Permission.Action... actions) throws Throwable {
112    connection.getAdmin().grant(
113      new UserPermission(userName, Permission.newBuilder(namespace).withActions(actions).build()),
114      mergeExistingPermissions);
115  }
116
117  /**
118   * Grants permission on the specified namespace for the specified user. If permissions on the
119   * specified namespace exists, later granted permissions will override previous granted
120   * permissions.
121   * @param connection The Connection instance to use
122   * @param namespace  the namespace
123   * @param userName   the user name
124   * @param actions    the actions
125   */
126  public static void grant(Connection connection, final String namespace, final String userName,
127    final Permission.Action... actions) throws Throwable {
128    grant(connection, namespace, userName, true, actions);
129  }
130
131  /**
132   * Grant global permissions for the specified user.
133   * @param connection               The Connection instance to use
134   * @param userName                 the user name
135   * @param mergeExistingPermissions If set to false, later granted permissions will override
136   *                                 previous granted permissions. otherwise, it'll merge with
137   *                                 previous granted permissions.
138   * @param actions                  the actions
139   */
140  private static void grant(Connection connection, final String userName,
141    boolean mergeExistingPermissions, final Permission.Action... actions) throws Throwable {
142    connection.getAdmin().grant(
143      new UserPermission(userName, Permission.newBuilder().withActions(actions).build()),
144      mergeExistingPermissions);
145  }
146
147  /**
148   * Grant global permissions for the specified user. If permissions for the specified user exists,
149   * later granted permissions will override previous granted permissions.
150   */
151  public static void grant(Connection connection, final String userName,
152    final Permission.Action... actions) throws Throwable {
153    grant(connection, userName, true, actions);
154  }
155
156  public static boolean isAccessControllerRunning(Connection connection)
157    throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
158    try (Admin admin = connection.getAdmin()) {
159      return admin.isTableAvailable(ACL_TABLE_NAME);
160    }
161  }
162
163  /**
164   * Revokes the permission on the table
165   * @param connection The Connection instance to use
166   */
167  public static void revoke(Connection connection, final TableName tableName, final String username,
168    final byte[] family, final byte[] qualifier, final Permission.Action... actions)
169    throws Throwable {
170    connection.getAdmin().revoke(new UserPermission(username, Permission.newBuilder(tableName)
171      .withFamily(family).withQualifier(qualifier).withActions(actions).build()));
172  }
173
174  /**
175   * Revokes the permission on the namespace for the specified user.
176   * @param connection The Connection instance to use
177   */
178  public static void revoke(Connection connection, final String namespace, final String userName,
179    final Permission.Action... actions) throws Throwable {
180    connection.getAdmin().revoke(
181      new UserPermission(userName, Permission.newBuilder(namespace).withActions(actions).build()));
182  }
183
184  /**
185   * Revoke global permissions for the specified user.
186   * @param connection The Connection instance to use
187   */
188  public static void revoke(Connection connection, final String userName,
189    final Permission.Action... actions) throws Throwable {
190    connection.getAdmin()
191      .revoke(new UserPermission(userName, Permission.newBuilder().withActions(actions).build()));
192  }
193
194  /**
195   * List all the userPermissions matching the given pattern. If pattern is null, the behavior is
196   * dependent on whether user has global admin privileges or not. If yes, the global permissions
197   * along with the list of superusers would be returned. Else, no rows get returned.
198   * @param connection The Connection instance to use
199   * @param tableRegex The regular expression string to match against
200   * @return List of UserPermissions
201   */
202  public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex)
203    throws Throwable {
204    return getUserPermissions(connection, tableRegex, HConstants.EMPTY_STRING);
205  }
206
207  /**
208   * List all the userPermissions matching the given table pattern and user name.
209   * @param connection Connection
210   * @param tableRegex The regular expression string to match against
211   * @param userName   User name, if empty then all user permissions will be retrieved.
212   * @return List of UserPermissions
213   * @throws Throwable on failure
214   */
215  public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex,
216    String userName) throws Throwable {
217    List<UserPermission> permList = new ArrayList<>();
218    try (Admin admin = connection.getAdmin()) {
219      if (tableRegex == null || tableRegex.isEmpty()) {
220        permList = admin.getUserPermissions(
221          GetUserPermissionsRequest.newBuilder().withUserName(userName).build());
222      } else if (tableRegex.charAt(0) == '@') { // Namespaces
223        String namespaceRegex = tableRegex.substring(1);
224        for (NamespaceDescriptor nsds : admin.listNamespaceDescriptors()) { // Read out all
225                                                                            // namespaces
226          String namespace = nsds.getName();
227          if (namespace.matches(namespaceRegex)) { // Match the given namespace regex?
228            permList.addAll(admin.getUserPermissions(
229              GetUserPermissionsRequest.newBuilder(namespace).withUserName(userName).build()));
230          }
231        }
232      } else { // Tables
233        List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
234        for (TableDescriptor htd : htds) {
235          permList.addAll(admin.getUserPermissions(GetUserPermissionsRequest
236            .newBuilder(htd.getTableName()).withUserName(userName).build()));
237        }
238      }
239    }
240    return permList;
241  }
242
243  /**
244   * List all the userPermissions matching the given table pattern and column family.
245   * @param connection   Connection
246   * @param tableRegex   The regular expression string to match against. It shouldn't be null, empty
247   *                     or a namespace regular expression.
248   * @param columnFamily Column family
249   * @return List of UserPermissions
250   * @throws Throwable on failure
251   */
252  public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex,
253    byte[] columnFamily) throws Throwable {
254    return getUserPermissions(connection, tableRegex, columnFamily, null, HConstants.EMPTY_STRING);
255  }
256
257  /**
258   * List all the userPermissions matching the given table pattern, column family and user name.
259   * @param connection   Connection
260   * @param tableRegex   The regular expression string to match against. It shouldn't be null, empty
261   *                     or a namespace regular expression.
262   * @param columnFamily Column family
263   * @param userName     User name, if empty then all user permissions will be retrieved.
264   * @return List of UserPermissions
265   * @throws Throwable on failure
266   */
267  public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex,
268    byte[] columnFamily, String userName) throws Throwable {
269    return getUserPermissions(connection, tableRegex, columnFamily, null, userName);
270  }
271
272  /**
273   * List all the userPermissions matching the given table pattern, column family and column
274   * qualifier.
275   * @param connection      Connection
276   * @param tableRegex      The regular expression string to match against. It shouldn't be null,
277   *                        empty or a namespace regular expression.
278   * @param columnFamily    Column family
279   * @param columnQualifier Column qualifier
280   * @return List of UserPermissions
281   * @throws Throwable on failure
282   */
283  public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex,
284    byte[] columnFamily, byte[] columnQualifier) throws Throwable {
285    return getUserPermissions(connection, tableRegex, columnFamily, columnQualifier,
286      HConstants.EMPTY_STRING);
287  }
288
289  /**
290   * List all the userPermissions matching the given table pattern, column family and column
291   * qualifier.
292   * @param connection      Connection
293   * @param tableRegex      The regular expression string to match against. It shouldn't be null,
294   *                        empty or a namespace regular expression.
295   * @param columnFamily    Column family
296   * @param columnQualifier Column qualifier
297   * @param userName        User name, if empty then all user permissions will be retrieved.
298   * @return List of UserPermissions
299   * @throws Throwable on failure
300   */
301  public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex,
302    byte[] columnFamily, byte[] columnQualifier, String userName) throws Throwable {
303    if (tableRegex == null || tableRegex.isEmpty() || tableRegex.charAt(0) == '@') {
304      throw new IllegalArgumentException("Table name can't be null or empty or a namespace.");
305    }
306    List<UserPermission> permList = new ArrayList<UserPermission>();
307    try (Admin admin = connection.getAdmin()) {
308      List<TableDescriptor> htds = admin.listTableDescriptors(Pattern.compile(tableRegex), true);
309      // Retrieve table permissions
310      for (TableDescriptor htd : htds) {
311        permList.addAll(admin.getUserPermissions(
312          GetUserPermissionsRequest.newBuilder(htd.getTableName()).withFamily(columnFamily)
313            .withQualifier(columnQualifier).withUserName(userName).build()));
314      }
315    }
316    return permList;
317  }
318
319  /**
320   * Validates whether specified user has permission to perform actions on the mentioned table,
321   * column family or column qualifier.
322   * @param connection      Connection
323   * @param tableName       Table name, it shouldn't be null or empty.
324   * @param columnFamily    The column family. Optional argument, can be empty. If empty then
325   *                        validation will happen at table level.
326   * @param columnQualifier The column qualifier. Optional argument, can be empty. If empty then
327   *                        validation will happen at table and column family level. columnQualifier
328   *                        will not be considered if columnFamily is passed as null or empty.
329   * @param userName        User name, it shouldn't be null or empty.
330   * @param actions         Actions
331   * @return true if access allowed to the specified user, otherwise false.
332   * @throws Throwable on failure
333   */
334  public static boolean hasPermission(Connection connection, String tableName, String columnFamily,
335    String columnQualifier, String userName, Permission.Action... actions) throws Throwable {
336    return hasPermission(connection, tableName, Bytes.toBytes(columnFamily),
337      Bytes.toBytes(columnQualifier), userName, actions);
338  }
339
340  /**
341   * Validates whether specified user has permission to perform actions on the mentioned table,
342   * column family or column qualifier.
343   * @param connection      Connection
344   * @param tableName       Table name, it shouldn't be null or empty.
345   * @param columnFamily    The column family. Optional argument, can be empty. If empty then
346   *                        validation will happen at table level.
347   * @param columnQualifier The column qualifier. Optional argument, can be empty. If empty then
348   *                        validation will happen at table and column family level. columnQualifier
349   *                        will not be considered if columnFamily is passed as null or empty.
350   * @param userName        User name, it shouldn't be null or empty.
351   * @param actions         Actions
352   * @return true if access allowed to the specified user, otherwise false.
353   * @throws Throwable on failure
354   */
355  public static boolean hasPermission(Connection connection, String tableName, byte[] columnFamily,
356    byte[] columnQualifier, String userName, Permission.Action... actions) throws Throwable {
357    if (StringUtils.isEmpty(tableName) || StringUtils.isEmpty(userName)) {
358      throw new IllegalArgumentException("Table and user name can't be null or empty.");
359    }
360    List<Permission> permissions = new ArrayList<>(1);
361    permissions.add(Permission.newBuilder(TableName.valueOf(tableName)).withFamily(columnFamily)
362      .withQualifier(columnQualifier).withActions(actions).build());
363    return connection.getAdmin().hasUserPermissions(userName, permissions).get(0);
364  }
365}