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    package org.apache.hadoop.security.token.delegation.web;
019    
020    import org.apache.hadoop.classification.InterfaceAudience;
021    import org.apache.hadoop.classification.InterfaceStability;
022    import org.apache.hadoop.security.SecurityUtil;
023    import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
024    import org.apache.hadoop.security.authentication.client.AuthenticationException;
025    import org.apache.hadoop.security.authentication.client.Authenticator;
026    import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
027    import org.apache.hadoop.security.token.Token;
028    import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
029    import org.apache.hadoop.util.HttpExceptionUtils;
030    import org.codehaus.jackson.map.ObjectMapper;
031    import org.slf4j.Logger;
032    import org.slf4j.LoggerFactory;
033    
034    import java.io.IOException;
035    import java.net.HttpURLConnection;
036    import java.net.InetSocketAddress;
037    import java.net.URL;
038    import java.net.URLEncoder;
039    import java.util.HashMap;
040    import 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
048    public 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    }