HBASE-25611 ExportSnapshot chmod flag uses value as decimal (#3003)
[hbase.git] / hbase-common / src / main / java / org / apache / hadoop / hbase / security / UserProvider.java
blob0c054ceaaa286a96db15080acee7bafe3a01a794
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;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.fs.CommonConfigurationKeys;
28 import org.apache.hadoop.hbase.BaseConfigurable;
29 import org.apache.hadoop.security.Groups;
30 import org.apache.hadoop.security.UserGroupInformation;
31 import org.apache.hadoop.util.ReflectionUtils;
32 import org.apache.yetus.audience.InterfaceAudience;
34 import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
35 import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
36 import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
37 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListenableFuture;
38 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListeningExecutorService;
39 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.MoreExecutors;
40 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
42 /**
43 * Provide an instance of a user. Allows custom {@link User} creation.
45 @InterfaceAudience.Private
46 public class UserProvider extends BaseConfigurable {
48 private static final String USER_PROVIDER_CONF_KEY = "hbase.client.userprovider.class";
49 private static final ListeningExecutorService executor = MoreExecutors.listeningDecorator(
50 Executors.newScheduledThreadPool(
52 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("group-cache-%d").build()));
54 private LoadingCache<String, String[]> groupCache = null;
56 static Groups groups = Groups.getUserToGroupsMappingService();
58 public static Groups getGroups() {
59 return groups;
62 public static void setGroups(Groups groups) {
63 UserProvider.groups = groups;
66 @Override
67 public void setConf(final Configuration conf) {
68 super.setConf(conf);
70 synchronized (UserProvider.class) {
71 if (!(groups instanceof User.TestingGroups)) {
72 groups = Groups.getUserToGroupsMappingService(conf);
76 long cacheTimeout =
77 getConf().getLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS,
78 CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS_DEFAULT) * 1000;
80 this.groupCache = CacheBuilder.newBuilder()
81 // This is the same timeout that hadoop uses. So we'll follow suit.
82 .refreshAfterWrite(cacheTimeout, TimeUnit.MILLISECONDS)
83 .expireAfterWrite(10 * cacheTimeout, TimeUnit.MILLISECONDS)
84 // Set concurrency level equal to the default number of handlers that
85 // the simple handler spins up.
86 .concurrencyLevel(20)
87 // create the loader
88 // This just delegates to UGI.
89 .build(new CacheLoader<String, String[]>() {
91 // Since UGI's don't hash based on the user id
92 // The cache needs to be keyed on the same thing that Hadoop's Groups class
93 // uses. So this cache uses shortname.
94 @Override
95 public String[] load(String ugi) throws Exception {
96 return getGroupStrings(ugi);
99 private String[] getGroupStrings(String ugi) {
100 try {
101 Set<String> result = new LinkedHashSet<>(groups.getGroups(ugi));
102 return result.toArray(new String[result.size()]);
103 } catch (Exception e) {
104 return new String[0];
108 // Provide the reload function that uses the executor thread.
109 @Override
110 public ListenableFuture<String[]> reload(final String k, String[] oldValue)
111 throws Exception {
113 return executor.submit(new Callable<String[]>() {
114 @Override
115 public String[] call() throws Exception {
116 return getGroupStrings(k);
124 * Instantiate the {@link UserProvider} specified in the configuration and set the passed
125 * configuration via {@link UserProvider#setConf(Configuration)}
126 * @param conf to read and set on the created {@link UserProvider}
127 * @return a {@link UserProvider} ready for use.
129 public static UserProvider instantiate(Configuration conf) {
130 Class<? extends UserProvider> clazz =
131 conf.getClass(USER_PROVIDER_CONF_KEY, UserProvider.class, UserProvider.class);
132 return ReflectionUtils.newInstance(clazz, conf);
136 * Set the {@link UserProvider} in the given configuration that should be instantiated
137 * @param conf to update
138 * @param provider class of the provider to set
140 public static void setUserProviderForTesting(Configuration conf,
141 Class<? extends UserProvider> provider) {
142 conf.set(USER_PROVIDER_CONF_KEY, provider.getName());
146 * @return the userName for the current logged-in user.
147 * @throws IOException if the underlying user cannot be obtained
149 public String getCurrentUserName() throws IOException {
150 User user = getCurrent();
151 return user == null ? null : user.getName();
155 * @return <tt>true</tt> if security is enabled, <tt>false</tt> otherwise
157 public boolean isHBaseSecurityEnabled() {
158 return User.isHBaseSecurityEnabled(this.getConf());
162 * @return whether or not Kerberos authentication is configured for Hadoop. For non-secure Hadoop,
163 * this always returns <code>false</code>. For secure Hadoop, it will return the value
164 * from {@code UserGroupInformation.isSecurityEnabled()}.
166 public boolean isHadoopSecurityEnabled() {
167 return User.isSecurityEnabled();
171 * In secure environment, if a user specified his keytab and principal,
172 * a hbase client will try to login with them. Otherwise, hbase client will try to obtain
173 * ticket(through kinit) from system.
175 public boolean shouldLoginFromKeytab() {
176 return User.shouldLoginFromKeytab(this.getConf());
180 * @return the current user within the current execution context
181 * @throws IOException if the user cannot be loaded
183 public User getCurrent() throws IOException {
184 return User.getCurrent();
188 * Wraps an underlying {@code UserGroupInformation} instance.
189 * @param ugi The base Hadoop user
190 * @return User
192 public User create(UserGroupInformation ugi) {
193 if (ugi == null) {
194 return null;
196 return new User.SecureHadoopUser(ugi, groupCache);
200 * Log in the current process using the given configuration keys for the credential file and login
201 * principal. It is for SPN(Service Principal Name) login. SPN should be this format,
202 * servicename/fully.qualified.domain.name@REALM.
203 * <p>
204 * <strong>This is only applicable when running on secure Hadoop</strong> -- see
205 * org.apache.hadoop.security.SecurityUtil#login(Configuration,String,String,String). On regular
206 * Hadoop (without security features), this will safely be ignored.
207 * </p>
208 * @param fileConfKey Property key used to configure path to the credential file
209 * @param principalConfKey Property key used to configure login principal
210 * @param localhost Current hostname to use in any credentials
211 * @throws IOException underlying exception from SecurityUtil.login() call
213 public void login(String fileConfKey, String principalConfKey, String localhost)
214 throws IOException {
215 User.login(getConf(), fileConfKey, principalConfKey, localhost);
219 * Login with given keytab and principal. This can be used for both SPN(Service Principal Name)
220 * and UPN(User Principal Name) which format should be clientname@REALM.
221 * @param fileConfKey config name for client keytab
222 * @param principalConfKey config name for client principal
223 * @throws IOException underlying exception from UserGroupInformation.loginUserFromKeytab
225 public void login(String fileConfKey, String principalConfKey) throws IOException {
226 User.login(getConf().get(fileConfKey), getConf().get(principalConfKey));