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