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