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.security.token.delegation.web;
019
020import org.apache.hadoop.classification.InterfaceAudience;
021import org.apache.hadoop.classification.InterfaceStability;
022import org.apache.hadoop.security.SecurityUtil;
023import org.apache.hadoop.security.UserGroupInformation;
024import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
025import org.apache.hadoop.security.authentication.client.AuthenticationException;
026import org.apache.hadoop.security.authentication.client.Authenticator;
027import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
028import org.apache.hadoop.security.token.Token;
029import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
030import org.apache.hadoop.util.HttpExceptionUtils;
031import org.codehaus.jackson.map.ObjectMapper;
032import org.slf4j.Logger;
033import org.slf4j.LoggerFactory;
034
035import java.io.IOException;
036import java.net.HttpURLConnection;
037import java.net.InetSocketAddress;
038import java.net.URL;
039import java.net.URLEncoder;
040import java.util.HashMap;
041import java.util.Map;
042
043/**
044 * {@link Authenticator} wrapper that enhances an {@link Authenticator} with
045 * Delegation Token support.
046 */
047@InterfaceAudience.Public
048@InterfaceStability.Evolving
049public abstract class DelegationTokenAuthenticator implements Authenticator {
050  private static Logger LOG = 
051      LoggerFactory.getLogger(DelegationTokenAuthenticator.class);
052  
053  private static final String CONTENT_TYPE = "Content-Type";
054  private static final String APPLICATION_JSON_MIME = "application/json";
055
056  private static final String HTTP_GET = "GET";
057  private static final String HTTP_PUT = "PUT";
058
059  public static final String OP_PARAM = "op";
060
061  public static final String DELEGATION_TOKEN_HEADER =
062      "X-Hadoop-Delegation-Token";
063
064  public static final String DELEGATION_PARAM = "delegation";
065  public static final String TOKEN_PARAM = "token";
066  public static final String RENEWER_PARAM = "renewer";
067  public static final String DELEGATION_TOKEN_JSON = "Token";
068  public static final String DELEGATION_TOKEN_URL_STRING_JSON = "urlString";
069  public static final String RENEW_DELEGATION_TOKEN_JSON = "long";
070
071  /**
072   * DelegationToken operations.
073   */
074  @InterfaceAudience.Private
075  public static enum DelegationTokenOperation {
076    GETDELEGATIONTOKEN(HTTP_GET, true),
077    RENEWDELEGATIONTOKEN(HTTP_PUT, true),
078    CANCELDELEGATIONTOKEN(HTTP_PUT, false);
079
080    private String httpMethod;
081    private boolean requiresKerberosCredentials;
082
083    private DelegationTokenOperation(String httpMethod,
084        boolean requiresKerberosCredentials) {
085      this.httpMethod = httpMethod;
086      this.requiresKerberosCredentials = requiresKerberosCredentials;
087    }
088
089    public String getHttpMethod() {
090      return httpMethod;
091    }
092
093    public boolean requiresKerberosCredentials() {
094      return requiresKerberosCredentials;
095    }
096  }
097
098  private Authenticator authenticator;
099  private ConnectionConfigurator connConfigurator;
100
101  public DelegationTokenAuthenticator(Authenticator authenticator) {
102    this.authenticator = authenticator;
103  }
104
105  @Override
106  public void setConnectionConfigurator(ConnectionConfigurator configurator) {
107    authenticator.setConnectionConfigurator(configurator);
108    connConfigurator = configurator;
109  }
110
111  private boolean hasDelegationToken(URL url, AuthenticatedURL.Token token) {
112    boolean hasDt = false;
113    if (token instanceof DelegationTokenAuthenticatedURL.Token) {
114      hasDt = ((DelegationTokenAuthenticatedURL.Token) token).
115          getDelegationToken() != null;
116    }
117    if (!hasDt) {
118      String queryStr = url.getQuery();
119      hasDt = (queryStr != null) && queryStr.contains(DELEGATION_PARAM + "=");
120    }
121    return hasDt;
122  }
123
124  @Override
125  public void authenticate(URL url, AuthenticatedURL.Token token)
126      throws IOException, AuthenticationException {
127    if (!hasDelegationToken(url, token)) {
128      // check and renew TGT to handle potential expiration
129      UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
130      authenticator.authenticate(url, token);
131    }
132  }
133
134  /**
135   * Requests a delegation token using the configured <code>Authenticator</code>
136   * for authentication.
137   *
138   * @param url the URL to get the delegation token from. Only HTTP/S URLs are
139   * supported.
140   * @param token the authentication token being used for the user where the
141   * Delegation token will be stored.
142   * @param renewer the renewer user.
143   * @throws IOException if an IO error occurred.
144   * @throws AuthenticationException if an authentication exception occurred.
145   */
146  public Token<AbstractDelegationTokenIdentifier> getDelegationToken(URL url,
147      AuthenticatedURL.Token token, String renewer)
148      throws IOException, AuthenticationException {
149   return getDelegationToken(url, token, renewer, null);
150  }
151
152  /**
153   * Requests a delegation token using the configured <code>Authenticator</code>
154   * for authentication.
155   *
156   * @param url the URL to get the delegation token from. Only HTTP/S URLs are
157   * supported.
158   * @param token the authentication token being used for the user where the
159   * Delegation token will be stored.
160   * @param renewer the renewer user.
161   * @param doAsUser the user to do as, which will be the token owner.
162   * @throws IOException if an IO error occurred.
163   * @throws AuthenticationException if an authentication exception occurred.
164   */
165  public Token<AbstractDelegationTokenIdentifier> getDelegationToken(URL url,
166      AuthenticatedURL.Token token, String renewer, String doAsUser)
167      throws IOException, AuthenticationException {
168    Map json = doDelegationTokenOperation(url, token,
169        DelegationTokenOperation.GETDELEGATIONTOKEN, renewer, null, true,
170        doAsUser);
171    json = (Map) json.get(DELEGATION_TOKEN_JSON);
172    String tokenStr = (String) json.get(DELEGATION_TOKEN_URL_STRING_JSON);
173    Token<AbstractDelegationTokenIdentifier> dToken =
174        new Token<AbstractDelegationTokenIdentifier>();
175    dToken.decodeFromUrlString(tokenStr);
176    InetSocketAddress service = new InetSocketAddress(url.getHost(),
177        url.getPort());
178    SecurityUtil.setTokenService(dToken, service);
179    return dToken;
180  }
181
182  /**
183   * Renews a delegation token from the server end-point using the
184   * configured <code>Authenticator</code> for authentication.
185   *
186   * @param url the URL to renew the delegation token from. Only HTTP/S URLs are
187   * supported.
188   * @param token the authentication token with the Delegation Token to renew.
189   * @throws IOException if an IO error occurred.
190   * @throws AuthenticationException if an authentication exception occurred.
191   */
192  public long renewDelegationToken(URL url,
193      AuthenticatedURL.Token token,
194      Token<AbstractDelegationTokenIdentifier> dToken)
195      throws IOException, AuthenticationException {
196    return renewDelegationToken(url, token, dToken, null);
197  }
198
199  /**
200   * Renews a delegation token from the server end-point using the
201   * configured <code>Authenticator</code> for authentication.
202   *
203   * @param url the URL to renew the delegation token from. Only HTTP/S URLs are
204   * supported.
205   * @param token the authentication token with the Delegation Token to renew.
206   * @param doAsUser the user to do as, which will be the token owner.
207   * @throws IOException if an IO error occurred.
208   * @throws AuthenticationException if an authentication exception occurred.
209   */
210  public long renewDelegationToken(URL url,
211      AuthenticatedURL.Token token,
212      Token<AbstractDelegationTokenIdentifier> dToken, String doAsUser)
213      throws IOException, AuthenticationException {
214    Map json = doDelegationTokenOperation(url, token,
215        DelegationTokenOperation.RENEWDELEGATIONTOKEN, null, dToken, true,
216        doAsUser);
217    return (Long) json.get(RENEW_DELEGATION_TOKEN_JSON);
218  }
219
220  /**
221   * Cancels a delegation token from the server end-point. It does not require
222   * being authenticated by the configured <code>Authenticator</code>.
223   *
224   * @param url the URL to cancel the delegation token from. Only HTTP/S URLs
225   * are supported.
226   * @param token the authentication token with the Delegation Token to cancel.
227   * @throws IOException if an IO error occurred.
228   */
229  public void cancelDelegationToken(URL url,
230      AuthenticatedURL.Token token,
231      Token<AbstractDelegationTokenIdentifier> dToken)
232      throws IOException {
233    cancelDelegationToken(url, token, dToken, null);
234  }
235
236  /**
237   * Cancels a delegation token from the server end-point. It does not require
238   * being authenticated by the configured <code>Authenticator</code>.
239   *
240   * @param url the URL to cancel the delegation token from. Only HTTP/S URLs
241   * are supported.
242   * @param token the authentication token with the Delegation Token to cancel.
243   * @param doAsUser the user to do as, which will be the token owner.
244   * @throws IOException if an IO error occurred.
245   */
246  public void cancelDelegationToken(URL url,
247      AuthenticatedURL.Token token,
248      Token<AbstractDelegationTokenIdentifier> dToken, String doAsUser)
249      throws IOException {
250    try {
251      doDelegationTokenOperation(url, token,
252          DelegationTokenOperation.CANCELDELEGATIONTOKEN, null, dToken, false,
253          doAsUser);
254    } catch (AuthenticationException ex) {
255      throw new IOException("This should not happen: " + ex.getMessage(), ex);
256    }
257  }
258
259  private Map doDelegationTokenOperation(URL url,
260      AuthenticatedURL.Token token, DelegationTokenOperation operation,
261      String renewer, Token<?> dToken, boolean hasResponse, String doAsUser)
262      throws IOException, AuthenticationException {
263    Map ret = null;
264    Map<String, String> params = new HashMap<String, String>();
265    params.put(OP_PARAM, operation.toString());
266    if (renewer != null) {
267      params.put(RENEWER_PARAM, renewer);
268    }
269    if (dToken != null) {
270      params.put(TOKEN_PARAM, dToken.encodeToUrlString());
271    }
272    // proxyuser
273    if (doAsUser != null) {
274      params.put(DelegationTokenAuthenticatedURL.DO_AS,
275          URLEncoder.encode(doAsUser, "UTF-8"));
276    }
277    String urlStr = url.toExternalForm();
278    StringBuilder sb = new StringBuilder(urlStr);
279    String separator = (urlStr.contains("?")) ? "&" : "?";
280    for (Map.Entry<String, String> entry : params.entrySet()) {
281      sb.append(separator).append(entry.getKey()).append("=").
282          append(URLEncoder.encode(entry.getValue(), "UTF8"));
283      separator = "&";
284    }
285    url = new URL(sb.toString());
286    AuthenticatedURL aUrl = new AuthenticatedURL(this, connConfigurator);
287    HttpURLConnection conn = aUrl.openConnection(url, token);
288    conn.setRequestMethod(operation.getHttpMethod());
289    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
290    if (hasResponse) {
291      String contentType = conn.getHeaderField(CONTENT_TYPE);
292      contentType = (contentType != null) ? contentType.toLowerCase()
293                                          : null;
294      if (contentType != null &&
295          contentType.contains(APPLICATION_JSON_MIME)) {
296        try {
297          ObjectMapper mapper = new ObjectMapper();
298          ret = mapper.readValue(conn.getInputStream(), Map.class);
299        } catch (Exception ex) {
300          throw new AuthenticationException(String.format(
301              "'%s' did not handle the '%s' delegation token operation: %s",
302              url.getAuthority(), operation, ex.getMessage()), ex);
303        }
304      } else {
305        throw new AuthenticationException(String.format("'%s' did not " +
306                "respond with JSON to the '%s' delegation token operation",
307            url.getAuthority(), operation));
308      }
309    }
310    return ret;
311  }
312
313}