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