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 */ 018 019package org.apache.hadoop.hbase.security.token; 020 021import com.google.protobuf.ByteString; 022import com.google.protobuf.ServiceException; 023import java.io.IOException; 024import java.lang.reflect.UndeclaredThrowableException; 025import java.security.PrivilegedExceptionAction; 026import org.apache.hadoop.hbase.HConstants; 027import org.apache.hadoop.hbase.TableName; 028import org.apache.hadoop.hbase.client.Connection; 029import org.apache.hadoop.hbase.client.Table; 030import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; 031import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos; 032import org.apache.hadoop.hbase.security.User; 033import org.apache.hadoop.io.Text; 034import org.apache.hadoop.security.token.Token; 035import org.apache.yetus.audience.InterfaceAudience; 036import org.slf4j.Logger; 037import org.slf4j.LoggerFactory; 038import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 039 040/** 041 * Utility methods for obtaining authentication tokens, that do not require hbase-server. 042 */ 043@InterfaceAudience.Public 044public final class ClientTokenUtil { 045 private static final Logger LOG = LoggerFactory.getLogger(ClientTokenUtil.class); 046 047 // Set in TestClientTokenUtil via reflection 048 private static ServiceException injectedException; 049 050 private ClientTokenUtil() {} 051 052 private static void injectFault() throws ServiceException { 053 if (injectedException != null) { 054 throw injectedException; 055 } 056 } 057 058 /** 059 * Obtain and return an authentication token for the current user. 060 * @param conn The HBase cluster connection 061 * @throws IOException if a remote error or serialization problem occurs. 062 * @return the authentication token instance 063 */ 064 @InterfaceAudience.Private 065 public static Token<AuthenticationTokenIdentifier> obtainToken( 066 Connection conn) throws IOException { 067 Table meta = null; 068 try { 069 injectFault(); 070 071 meta = conn.getTable(TableName.META_TABLE_NAME); 072 CoprocessorRpcChannel rpcChannel = meta.coprocessorService( 073 HConstants.EMPTY_START_ROW); 074 AuthenticationProtos.AuthenticationService.BlockingInterface service = 075 AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel); 076 AuthenticationProtos.GetAuthenticationTokenResponse response = 077 service.getAuthenticationToken(null, 078 AuthenticationProtos.GetAuthenticationTokenRequest.getDefaultInstance()); 079 080 return toToken(response.getToken()); 081 } catch (ServiceException se) { 082 throw ProtobufUtil.handleRemoteException(se); 083 } finally { 084 if (meta != null) { 085 meta.close(); 086 } 087 } 088 } 089 090 /** 091 * Converts a Token instance (with embedded identifier) to the protobuf representation. 092 * 093 * @param token the Token instance to copy 094 * @return the protobuf Token message 095 */ 096 @InterfaceAudience.Private 097 static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) { 098 AuthenticationProtos.Token.Builder builder = AuthenticationProtos.Token.newBuilder(); 099 builder.setIdentifier(ByteString.copyFrom(token.getIdentifier())); 100 builder.setPassword(ByteString.copyFrom(token.getPassword())); 101 if (token.getService() != null) { 102 builder.setService(ByteString.copyFromUtf8(token.getService().toString())); 103 } 104 return builder.build(); 105 } 106 107 /** 108 * Converts a protobuf Token message back into a Token instance. 109 * 110 * @param proto the protobuf Token message 111 * @return the Token instance 112 */ 113 @InterfaceAudience.Private 114 static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) { 115 return new Token<>( 116 proto.hasIdentifier() ? proto.getIdentifier().toByteArray() : null, 117 proto.hasPassword() ? proto.getPassword().toByteArray() : null, 118 AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE, 119 proto.hasService() ? new Text(proto.getService().toStringUtf8()) : null); 120 } 121 122 /** 123 * Obtain and return an authentication token for the given user. 124 * @param conn The HBase cluster connection 125 * @param user The user to obtain a token for 126 * @return the authentication token instance 127 */ 128 @InterfaceAudience.Private 129 static Token<AuthenticationTokenIdentifier> obtainToken( 130 final Connection conn, User user) throws IOException, InterruptedException { 131 return user.runAs(new PrivilegedExceptionAction<Token<AuthenticationTokenIdentifier>>() { 132 @Override 133 public Token<AuthenticationTokenIdentifier> run() throws Exception { 134 return obtainToken(conn); 135 } 136 }); 137 } 138 139 /** 140 * Obtain an authentication token for the given user and add it to the 141 * user's credentials. 142 * @param conn The HBase cluster connection 143 * @param user The user for whom to obtain the token 144 * @throws IOException If making a remote call to the authentication service fails 145 * @throws InterruptedException If executing as the given user is interrupted 146 */ 147 public static void obtainAndCacheToken(final Connection conn, 148 User user) 149 throws IOException, InterruptedException { 150 try { 151 Token<AuthenticationTokenIdentifier> token = obtainToken(conn, user); 152 153 if (token == null) { 154 throw new IOException("No token returned for user " + user.getName()); 155 } 156 if (LOG.isDebugEnabled()) { 157 LOG.debug("Obtained token " + token.getKind().toString() + " for user " + 158 user.getName()); 159 } 160 user.addToken(token); 161 } catch (IOException | InterruptedException | RuntimeException e) { 162 throw e; 163 } catch (Exception e) { 164 throw new UndeclaredThrowableException(e, 165 "Unexpected exception obtaining token for user " + user.getName()); 166 } 167 } 168}