HBASE-23892 SecureTestCluster should allow its subclasses to pass their Class referen...
[hbase.git] / hbase-common / src / main / java / org / apache / hadoop / hbase / security / UserProvider.java
blobefa18fb9f5864d73c5d52684229e7507d4d806bf
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
10 * http://www.apache.org/licenses/LICENSE-2.0
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.
18 package org.apache.hadoop.hbase.security;
20 import java.io.IOException;
21 import java.util.LinkedHashSet;
22 import java.util.Set;
23 import java.util.concurrent.Callable;
24 import java.util.concurrent.Executors;
25 import java.util.concurrent.TimeUnit;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.fs.CommonConfigurationKeys;
29 import org.apache.hadoop.hbase.BaseConfigurable;
30 import org.apache.hadoop.security.Groups;
31 import org.apache.hadoop.security.UserGroupInformation;
32 import org.apache.hadoop.util.ReflectionUtils;
33 import org.apache.yetus.audience.InterfaceAudience;
35 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
36 import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
37 import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
38 import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
39 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListenableFuture;
40 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
41 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.MoreExecutors;
42 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
44 /**
45 * Provide an instance of a user. Allows custom {@link User} creation.
47 @InterfaceAudience.Private
48 public class UserProvider extends BaseConfigurable {
50 private static final String USER_PROVIDER_CONF_KEY = "hbase.client.userprovider.class";
51 private static final ListeningExecutorService executor = MoreExecutors.listeningDecorator(
52 Executors.newScheduledThreadPool(
54 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("group-cache-%d").build()));
56 private LoadingCache<String, String[]> groupCache = null;
58 static Groups groups = Groups.getUserToGroupsMappingService();
60 @VisibleForTesting
61 public static Groups getGroups() {
62 return groups;
65 public static void setGroups(Groups groups) {
66 UserProvider.groups = groups;
69 @Override
70 public void setConf(final Configuration conf) {
71 super.setConf(conf);
73 synchronized (UserProvider.class) {
74 if (!(groups instanceof User.TestingGroups)) {
75 groups = Groups.getUserToGroupsMappingService(conf);
79 long cacheTimeout =
80 getConf().getLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS,
81 CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS_DEFAULT) * 1000;
83 this.groupCache = CacheBuilder.newBuilder()
84 // This is the same timeout that hadoop uses. So we'll follow suit.
85 .refreshAfterWrite(cacheTimeout, TimeUnit.MILLISECONDS)
86 .expireAfterWrite(10 * cacheTimeout, TimeUnit.MILLISECONDS)
87 // Set concurrency level equal to the default number of handlers that
88 // the simple handler spins up.
89 .concurrencyLevel(20)
90 // create the loader
91 // This just delegates to UGI.
92 .build(new CacheLoader<String, String[]>() {
94 // Since UGI's don't hash based on the user id
95 // The cache needs to be keyed on the same thing that Hadoop's Groups class
96 // uses. So this cache uses shortname.
97 @Override
98 public String[] load(String ugi) throws Exception {
99 return getGroupStrings(ugi);
102 private String[] getGroupStrings(String ugi) {
103 try {
104 Set<String> result = new LinkedHashSet<>(groups.getGroups(ugi));
105 return result.toArray(new String[result.size()]);
106 } catch (Exception e) {
107 return new String[0];
111 // Provide the reload function that uses the executor thread.
112 @Override
113 public ListenableFuture<String[]> reload(final String k, String[] oldValue)
114 throws Exception {
116 return executor.submit(new Callable<String[]>() {
117 @Override
118 public String[] call() throws Exception {
119 return getGroupStrings(k);
127 * Instantiate the {@link UserProvider} specified in the configuration and set the passed
128 * configuration via {@link UserProvider#setConf(Configuration)}
129 * @param conf to read and set on the created {@link UserProvider}
130 * @return a {@link UserProvider} ready for use.
132 public static UserProvider instantiate(Configuration conf) {
133 Class<? extends UserProvider> clazz =
134 conf.getClass(USER_PROVIDER_CONF_KEY, UserProvider.class, UserProvider.class);
135 return ReflectionUtils.newInstance(clazz, conf);
139 * Set the {@link UserProvider} in the given configuration that should be instantiated
140 * @param conf to update
141 * @param provider class of the provider to set
143 public static void setUserProviderForTesting(Configuration conf,
144 Class<? extends UserProvider> provider) {
145 conf.set(USER_PROVIDER_CONF_KEY, provider.getName());
149 * @return the userName for the current logged-in user.
150 * @throws IOException if the underlying user cannot be obtained
152 public String getCurrentUserName() throws IOException {
153 User user = getCurrent();
154 return user == null ? null : user.getName();
158 * @return <tt>true</tt> if security is enabled, <tt>false</tt> otherwise
160 public boolean isHBaseSecurityEnabled() {
161 return User.isHBaseSecurityEnabled(this.getConf());
165 * @return whether or not Kerberos authentication is configured for Hadoop. For non-secure Hadoop,
166 * this always returns <code>false</code>. For secure Hadoop, it will return the value
167 * from {@code UserGroupInformation.isSecurityEnabled()}.
169 public boolean isHadoopSecurityEnabled() {
170 return User.isSecurityEnabled();
174 * In secure environment, if a user specified his keytab and principal,
175 * a hbase client will try to login with them. Otherwise, hbase client will try to obtain
176 * ticket(through kinit) from system.
178 public boolean shouldLoginFromKeytab() {
179 return User.shouldLoginFromKeytab(this.getConf());
183 * @return the current user within the current execution context
184 * @throws IOException if the user cannot be loaded
186 public User getCurrent() throws IOException {
187 return User.getCurrent();
191 * Wraps an underlying {@code UserGroupInformation} instance.
192 * @param ugi The base Hadoop user
193 * @return User
195 public User create(UserGroupInformation ugi) {
196 if (ugi == null) {
197 return null;
199 return new User.SecureHadoopUser(ugi, groupCache);
203 * Log in the current process using the given configuration keys for the credential file and login
204 * principal. It is for SPN(Service Principal Name) login. SPN should be this format,
205 * servicename/fully.qualified.domain.name@REALM.
206 * <p>
207 * <strong>This is only applicable when running on secure Hadoop</strong> -- see
208 * org.apache.hadoop.security.SecurityUtil#login(Configuration,String,String,String). On regular
209 * Hadoop (without security features), this will safely be ignored.
210 * </p>
211 * @param fileConfKey Property key used to configure path to the credential file
212 * @param principalConfKey Property key used to configure login principal
213 * @param localhost Current hostname to use in any credentials
214 * @throws IOException underlying exception from SecurityUtil.login() call
216 public void login(String fileConfKey, String principalConfKey, String localhost)
217 throws IOException {
218 User.login(getConf(), fileConfKey, principalConfKey, localhost);
222 * Login with given keytab and principal. This can be used for both SPN(Service Principal Name)
223 * and UPN(User Principal Name) which format should be clientname@REALM.
224 * @param fileConfKey config name for client keytab
225 * @param principalConfKey config name for client principal
226 * @throws IOException underlying exception from UserGroupInformation.loginUserFromKeytab
228 public void login(String fileConfKey, String principalConfKey) throws IOException {
229 User.login(getConf().get(fileConfKey), getConf().get(principalConfKey));