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; 019 020import java.io.IOException; 021import java.util.LinkedHashSet; 022import java.util.Set; 023import java.util.concurrent.Callable; 024import java.util.concurrent.Executors; 025import java.util.concurrent.TimeUnit; 026 027import org.apache.hadoop.conf.Configuration; 028import org.apache.hadoop.fs.CommonConfigurationKeys; 029import org.apache.hadoop.hbase.BaseConfigurable; 030import org.apache.hadoop.security.Groups; 031import org.apache.hadoop.security.UserGroupInformation; 032import org.apache.hadoop.util.ReflectionUtils; 033import org.apache.yetus.audience.InterfaceAudience; 034 035import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 036import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder; 037import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader; 038import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache; 039import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListenableFuture; 040import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; 041import org.apache.hbase.thirdparty.com.google.common.util.concurrent.MoreExecutors; 042import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 043 044/** 045 * Provide an instance of a user. Allows custom {@link User} creation. 046 */ 047@InterfaceAudience.Private 048public class UserProvider extends BaseConfigurable { 049 050 private static final String USER_PROVIDER_CONF_KEY = "hbase.client.userprovider.class"; 051 private static final ListeningExecutorService executor = MoreExecutors.listeningDecorator( 052 Executors.newScheduledThreadPool( 053 1, 054 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("group-cache-%d").build())); 055 056 private LoadingCache<String, String[]> groupCache = null; 057 058 static Groups groups = Groups.getUserToGroupsMappingService(); 059 060 @VisibleForTesting 061 public static Groups getGroups() { 062 return groups; 063 } 064 065 public static void setGroups(Groups groups) { 066 UserProvider.groups = groups; 067 } 068 069 @Override 070 public void setConf(final Configuration conf) { 071 super.setConf(conf); 072 073 synchronized (UserProvider.class) { 074 if (!(groups instanceof User.TestingGroups)) { 075 groups = Groups.getUserToGroupsMappingService(conf); 076 } 077 } 078 079 long cacheTimeout = 080 getConf().getLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS, 081 CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS_DEFAULT) * 1000; 082 083 this.groupCache = CacheBuilder.newBuilder() 084 // This is the same timeout that hadoop uses. So we'll follow suit. 085 .refreshAfterWrite(cacheTimeout, TimeUnit.MILLISECONDS) 086 .expireAfterWrite(10 * cacheTimeout, TimeUnit.MILLISECONDS) 087 // Set concurrency level equal to the default number of handlers that 088 // the simple handler spins up. 089 .concurrencyLevel(20) 090 // create the loader 091 // This just delegates to UGI. 092 .build(new CacheLoader<String, String[]>() { 093 094 // Since UGI's don't hash based on the user id 095 // The cache needs to be keyed on the same thing that Hadoop's Groups class 096 // uses. So this cache uses shortname. 097 @Override 098 public String[] load(String ugi) throws Exception { 099 return getGroupStrings(ugi); 100 } 101 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]; 108 } 109 } 110 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 { 115 116 return executor.submit(new Callable<String[]>() { 117 @Override 118 public String[] call() throws Exception { 119 return getGroupStrings(k); 120 } 121 }); 122 } 123 }); 124 } 125 126 /** 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. 131 */ 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); 136 } 137 138 /** 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 142 */ 143 public static void setUserProviderForTesting(Configuration conf, 144 Class<? extends UserProvider> provider) { 145 conf.set(USER_PROVIDER_CONF_KEY, provider.getName()); 146 } 147 148 /** 149 * @return the userName for the current logged-in user. 150 * @throws IOException if the underlying user cannot be obtained 151 */ 152 public String getCurrentUserName() throws IOException { 153 User user = getCurrent(); 154 return user == null ? null : user.getName(); 155 } 156 157 /** 158 * @return <tt>true</tt> if security is enabled, <tt>false</tt> otherwise 159 */ 160 public boolean isHBaseSecurityEnabled() { 161 return User.isHBaseSecurityEnabled(this.getConf()); 162 } 163 164 /** 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()}. 168 */ 169 public boolean isHadoopSecurityEnabled() { 170 return User.isSecurityEnabled(); 171 } 172 173 /** 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. 177 */ 178 public boolean shouldLoginFromKeytab() { 179 return User.shouldLoginFromKeytab(this.getConf()); 180 } 181 182 /** 183 * @return the current user within the current execution context 184 * @throws IOException if the user cannot be loaded 185 */ 186 public User getCurrent() throws IOException { 187 return User.getCurrent(); 188 } 189 190 /** 191 * Wraps an underlying {@code UserGroupInformation} instance. 192 * @param ugi The base Hadoop user 193 * @return User 194 */ 195 public User create(UserGroupInformation ugi) { 196 if (ugi == null) { 197 return null; 198 } 199 return new User.SecureHadoopUser(ugi, groupCache); 200 } 201 202 /** 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 215 */ 216 public void login(String fileConfKey, String principalConfKey, String localhost) 217 throws IOException { 218 User.login(getConf(), fileConfKey, principalConfKey, localhost); 219 } 220 221 /** 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 227 */ 228 public void login(String fileConfKey, String principalConfKey) throws IOException { 229 User.login(getConf().get(fileConfKey), getConf().get(principalConfKey)); 230 } 231}