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