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