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.util.concurrent.CompletableFuture;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.hbase.client.AsyncConnection;
025import org.apache.hadoop.hbase.client.Connection;
026import org.apache.hadoop.hbase.client.ConnectionFactory;
027import org.apache.hadoop.hbase.security.User;
028import org.apache.hadoop.io.Text;
029import org.apache.hadoop.mapred.JobConf;
030import org.apache.hadoop.mapreduce.Job;
031import org.apache.hadoop.security.token.Token;
032import org.apache.yetus.audience.InterfaceAudience;
033import org.slf4j.Logger;
034import org.slf4j.LoggerFactory;
035
036import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos;
037
038/**
039 * Utility methods for obtaining authentication tokens.
040 */
041@InterfaceAudience.Public
042public class TokenUtil {
043  // This class is referenced indirectly by User out in common; instances are created by reflection
044  private static final Logger LOG = LoggerFactory.getLogger(TokenUtil.class);
045
046  /**
047   * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.AsyncConnection)}.
048   * @deprecated External users should not use this method, will be removed in 4.0.0. Please post on
049   *             the HBase dev mailing list if you need this method. Internal HBase code should use
050   *             {@link ClientTokenUtil} instead.
051   */
052  @Deprecated
053  public static CompletableFuture<Token<AuthenticationTokenIdentifier>>
054    obtainToken(AsyncConnection conn) {
055    return ClientTokenUtil.obtainToken(conn);
056  }
057
058  /**
059   * It was removed in HBase-2.0 but added again as spark code relies on this method to obtain
060   * delegation token
061   * @deprecated Since 2.0.0 and will be removed in 4.0.0.
062   */
063  @Deprecated
064  public static Token<AuthenticationTokenIdentifier> obtainToken(Configuration conf)
065    throws IOException {
066    try (Connection connection = ConnectionFactory.createConnection(conf)) {
067      return obtainToken(connection);
068    }
069  }
070
071  /**
072   * See {@link ClientTokenUtil#obtainToken(org.apache.hadoop.hbase.client.Connection)}.
073   * @deprecated External users should not use this method, will be removed in 4.0.0. Please post on
074   *             the HBase dev mailing list if you need this method. Internal HBase code should use
075   *             {@link ClientTokenUtil} instead.
076   */
077  @Deprecated
078  public static Token<AuthenticationTokenIdentifier> obtainToken(Connection conn)
079    throws IOException {
080    return ClientTokenUtil.obtainToken(conn);
081  }
082
083  /**
084   * See {@link ClientTokenUtil#toToken(Token)}.
085   * @deprecated External users should not use this method, will be removed in 4.0.0. Please post on
086   *             the HBase dev mailing list if you need this method. Internal HBase code should use
087   *             {@link ClientTokenUtil} instead.
088   */
089  @Deprecated
090  public static AuthenticationProtos.Token toToken(Token<AuthenticationTokenIdentifier> token) {
091    return ClientTokenUtil.toToken(token);
092  }
093
094  /**
095   * See {@link ClientTokenUtil#obtainToken(Connection, User)}.
096   * @deprecated External users should not use this method, will be removed in 4.0.0. Please post on
097   *             the HBase dev mailing list if you need this method. Internal HBase code should use
098   *             {@link ClientTokenUtil} instead.
099   */
100  @Deprecated
101  public static Token<AuthenticationTokenIdentifier> obtainToken(final Connection conn, User user)
102    throws IOException, InterruptedException {
103    return ClientTokenUtil.obtainToken(conn, user);
104  }
105
106  /**
107   * See {@link ClientTokenUtil#obtainAndCacheToken(Connection, User)}.
108   */
109  public static void obtainAndCacheToken(final Connection conn, User user)
110    throws IOException, InterruptedException {
111    ClientTokenUtil.obtainAndCacheToken(conn, user);
112  }
113
114  /**
115   * See
116   * {@link ClientTokenUtil#toToken(org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.Token)}.
117   * @deprecated External users should not use this method, will be removed in 4.0.0. Please post on
118   *             the HBase dev mailing list if you need this method. Internal HBase code should use
119   *             {@link ClientTokenUtil} instead.
120   */
121  @Deprecated
122  public static Token<AuthenticationTokenIdentifier> toToken(AuthenticationProtos.Token proto) {
123    return ClientTokenUtil.toToken(proto);
124  }
125
126  private static Text getClusterId(Token<AuthenticationTokenIdentifier> token) throws IOException {
127    return token.getService() != null ? token.getService() : new Text("default");
128  }
129
130  /**
131   * Obtain an authentication token on behalf of the given user and add it to the credentials for
132   * the given map reduce job.
133   * @param conn The HBase cluster connection
134   * @param user The user for whom to obtain the token
135   * @param job  The job instance in which the token should be stored
136   * @throws IOException          If making a remote call to the authentication service fails
137   * @throws InterruptedException If executing as the given user is interrupted
138   */
139  public static void obtainTokenForJob(final Connection conn, User user, Job job)
140    throws IOException, InterruptedException {
141    try {
142      Token<AuthenticationTokenIdentifier> token = ClientTokenUtil.obtainToken(conn, user);
143
144      if (token == null) {
145        throw new IOException("No token returned for user " + user.getName());
146      }
147      Text clusterId = getClusterId(token);
148      if (LOG.isDebugEnabled()) {
149        LOG.debug("Obtained token " + token.getKind().toString() + " for user " + user.getName()
150          + " on cluster " + clusterId.toString());
151      }
152      job.getCredentials().addToken(clusterId, token);
153    } catch (IOException ioe) {
154      throw ioe;
155    } catch (InterruptedException ie) {
156      throw ie;
157    } catch (RuntimeException re) {
158      throw re;
159    } catch (Exception e) {
160      throw new UndeclaredThrowableException(e,
161        "Unexpected exception obtaining token for user " + user.getName());
162    }
163  }
164
165  /**
166   * Obtain an authentication token on behalf of the given user and add it to the credentials for
167   * the given map reduce job.
168   * @param conn The HBase cluster connection
169   * @param user The user for whom to obtain the token
170   * @param job  The job configuration in which the token should be stored
171   * @throws IOException          If making a remote call to the authentication service fails
172   * @throws InterruptedException If executing as the given user is interrupted
173   */
174  public static void obtainTokenForJob(final Connection conn, final JobConf job, User user)
175    throws IOException, InterruptedException {
176    try {
177      Token<AuthenticationTokenIdentifier> token = ClientTokenUtil.obtainToken(conn, user);
178
179      if (token == null) {
180        throw new IOException("No token returned for user " + user.getName());
181      }
182      Text clusterId = getClusterId(token);
183      if (LOG.isDebugEnabled()) {
184        LOG.debug("Obtained token " + token.getKind().toString() + " for user " + user.getName()
185          + " on cluster " + clusterId.toString());
186      }
187      job.getCredentials().addToken(clusterId, token);
188    } catch (IOException ioe) {
189      throw ioe;
190    } catch (InterruptedException ie) {
191      throw ie;
192    } catch (RuntimeException re) {
193      throw re;
194    } catch (Exception e) {
195      throw new UndeclaredThrowableException(e,
196        "Unexpected exception obtaining token for user " + user.getName());
197    }
198  }
199
200  /**
201   * Checks for an authentication token for the given user, obtaining a new token if necessary, and
202   * adds it to the credentials for the given map reduce job.
203   * @param conn The HBase cluster connection
204   * @param user The user for whom to obtain the token
205   * @param job  The job configuration in which the token should be stored
206   * @throws IOException          If making a remote call to the authentication service fails
207   * @throws InterruptedException If executing as the given user is interrupted
208   */
209  public static void addTokenForJob(final Connection conn, final JobConf job, User user)
210    throws IOException, InterruptedException {
211
212    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
213    if (token == null) {
214      token = ClientTokenUtil.obtainToken(conn, user);
215    }
216    job.getCredentials().addToken(token.getService(), token);
217  }
218
219  /**
220   * Checks for an authentication token for the given user, obtaining a new token if necessary, and
221   * adds it to the credentials for the given map reduce job.
222   * @param conn The HBase cluster connection
223   * @param user The user for whom to obtain the token
224   * @param job  The job instance in which the token should be stored
225   * @throws IOException          If making a remote call to the authentication service fails
226   * @throws InterruptedException If executing as the given user is interrupted
227   */
228  public static void addTokenForJob(final Connection conn, User user, Job job)
229    throws IOException, InterruptedException {
230    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
231    if (token == null) {
232      token = ClientTokenUtil.obtainToken(conn, user);
233    }
234    job.getCredentials().addToken(token.getService(), token);
235  }
236
237  /**
238   * Checks if an authentication tokens exists for the connected cluster, obtaining one if needed
239   * and adding it to the user's credentials.
240   * @param conn The HBase cluster connection
241   * @param user The user for whom to obtain the token
242   * @throws IOException          If making a remote call to the authentication service fails
243   * @throws InterruptedException If executing as the given user is interrupted
244   * @return true if the token was added, false if it already existed
245   */
246  public static boolean addTokenIfMissing(Connection conn, User user)
247    throws IOException, InterruptedException {
248    Token<AuthenticationTokenIdentifier> token = getAuthToken(conn, user);
249    if (token == null) {
250      token = ClientTokenUtil.obtainToken(conn, user);
251      user.getUGI().addToken(token.getService(), token);
252      return true;
253    }
254    return false;
255  }
256
257  /**
258   * Get the authentication token of the user for the cluster specified in the configuration
259   * @return null if the user does not have the token, otherwise the auth token for the cluster.
260   */
261  private static Token<AuthenticationTokenIdentifier> getAuthToken(Connection conn, User user)
262    throws IOException {
263    final String clusterId = conn.getClusterId();
264    if (clusterId == null) {
265      throw new IOException("Failed to get cluster ID");
266    }
267    return new AuthenticationTokenSelector().selectToken(new Text(clusterId), user.getTokens());
268  }
269}