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