View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.thrift;
20  
21  import java.io.IOException;
22  import java.security.PrivilegedExceptionAction;
23  
24  import javax.servlet.ServletException;
25  import javax.servlet.http.HttpServletRequest;
26  import javax.servlet.http.HttpServletResponse;
27  
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.conf.Configuration;
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.hbase.security.SecurityUtil;
33  import org.apache.hadoop.hbase.util.Base64;
34  import org.apache.hadoop.security.UserGroupInformation;
35  import org.apache.hadoop.security.authorize.AuthorizationException;
36  import org.apache.hadoop.security.authorize.ProxyUsers;
37  import org.apache.thrift.TProcessor;
38  import org.apache.thrift.protocol.TProtocolFactory;
39  import org.apache.thrift.server.TServlet;
40  import org.ietf.jgss.GSSContext;
41  import org.ietf.jgss.GSSCredential;
42  import org.ietf.jgss.GSSException;
43  import org.ietf.jgss.GSSManager;
44  import org.ietf.jgss.GSSName;
45  import org.ietf.jgss.Oid;
46  
47  /**
48   * Thrift Http Servlet is used for performing Kerberos authentication if security is enabled and
49   * also used for setting the user specified in "doAs" parameter.
50   */
51  @InterfaceAudience.Private
52  public class ThriftHttpServlet extends TServlet {
53    private static final long serialVersionUID = 1L;
54    private static final Log LOG = LogFactory.getLog(ThriftHttpServlet.class.getName());
55    private transient final UserGroupInformation realUser;
56    private transient final Configuration conf;
57    private final boolean securityEnabled;
58    private final boolean doAsEnabled;
59    private transient ThriftServerRunner.HBaseHandler hbaseHandler;
60  
61    // HTTP Header related constants.
62    public static final String WWW_AUTHENTICATE = "WWW-Authenticate";
63    public static final String AUTHORIZATION = "Authorization";
64    public static final String NEGOTIATE = "Negotiate";
65  
66    public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory,
67        UserGroupInformation realUser, Configuration conf, ThriftServerRunner.HBaseHandler
68        hbaseHandler, boolean securityEnabled, boolean doAsEnabled) {
69      super(processor, protocolFactory);
70      this.realUser = realUser;
71      this.conf = conf;
72      this.hbaseHandler = hbaseHandler;
73      this.securityEnabled = securityEnabled;
74      this.doAsEnabled = doAsEnabled;
75    }
76  
77    @Override
78    protected void doPost(HttpServletRequest request, HttpServletResponse response)
79        throws ServletException, IOException {
80      String effectiveUser = request.getRemoteUser();
81      if (securityEnabled) {
82        try {
83          // As Thrift HTTP transport doesn't support SPNEGO yet (THRIFT-889),
84          // Kerberos authentication is being done at servlet level.
85          final RemoteUserIdentity identity = doKerberosAuth(request);
86          effectiveUser = identity.principal;
87          // It is standard for client applications expect this header.
88          // Please see http://tools.ietf.org/html/rfc4559 for more details.
89          response.addHeader(WWW_AUTHENTICATE,  NEGOTIATE + " " + identity.outToken);
90        } catch (HttpAuthenticationException e) {
91          LOG.error("Kerberos Authentication failed", e);
92          // Send a 401 to the client
93          response.setStatus(HttpServletResponse.SC_UNAUTHORIZED);
94          response.addHeader(WWW_AUTHENTICATE, NEGOTIATE);
95          response.getWriter().println("Authentication Error: " + e.getMessage());
96          return;
97        }
98      }
99      String doAsUserFromQuery = request.getHeader("doAs");
100     if(effectiveUser == null) {
101       effectiveUser = realUser.getShortUserName();
102     }
103     if (doAsUserFromQuery != null) {
104       if (!doAsEnabled) {
105         throw new ServletException("Support for proxyuser is not configured");
106       }
107       // The authenticated remote user is attempting to perform 'doAs' proxy user.
108       UserGroupInformation remoteUser = UserGroupInformation.createRemoteUser(effectiveUser);
109       // create and attempt to authorize a proxy user (the client is attempting
110       // to do proxy user)
111       UserGroupInformation ugi = UserGroupInformation.createProxyUser(doAsUserFromQuery,
112           remoteUser);
113       // validate the proxy user authorization
114       try {
115         ProxyUsers.authorize(ugi, request.getRemoteAddr(), conf);
116       } catch (AuthorizationException e) {
117         throw new ServletException(e.getMessage());
118       }
119       effectiveUser = doAsUserFromQuery;
120     }
121     hbaseHandler.setEffectiveUser(effectiveUser);
122     super.doPost(request, response);
123   }
124 
125   /**
126    * Do the GSS-API kerberos authentication.
127    * We already have a logged in subject in the form of serviceUGI,
128    * which GSS-API will extract information from.
129    */
130   private RemoteUserIdentity doKerberosAuth(HttpServletRequest request)
131       throws HttpAuthenticationException {
132     HttpKerberosServerAction action = new HttpKerberosServerAction(request, realUser);
133     try {
134       String principal = realUser.doAs(action);
135       return new RemoteUserIdentity(principal, action.outToken);
136     } catch (Exception e) {
137       LOG.error("Failed to perform authentication");
138       throw new HttpAuthenticationException(e);
139     }
140   }
141 
142   /**
143    * Basic "struct" class to hold the final base64-encoded, authenticated GSSAPI token
144    * for the user with the given principal talking to the Thrift server.
145    */
146   private static class RemoteUserIdentity {
147     final String outToken;
148     final String principal;
149 
150     RemoteUserIdentity(String principal, String outToken) {
151       this.principal = principal;
152       this.outToken = outToken;
153     }
154   }
155 
156   private static class HttpKerberosServerAction implements PrivilegedExceptionAction<String> {
157     HttpServletRequest request;
158     UserGroupInformation serviceUGI;
159     String outToken = null;
160     HttpKerberosServerAction(HttpServletRequest request, UserGroupInformation serviceUGI) {
161       this.request = request;
162       this.serviceUGI = serviceUGI;
163     }
164 
165     @Override
166     public String run() throws HttpAuthenticationException {
167       // Get own Kerberos credentials for accepting connection
168       GSSManager manager = GSSManager.getInstance();
169       GSSContext gssContext = null;
170       String serverPrincipal = SecurityUtil.getPrincipalWithoutRealm(serviceUGI.getUserName());
171       try {
172         // This Oid for Kerberos GSS-API mechanism.
173         Oid kerberosMechOid = new Oid("1.2.840.113554.1.2.2");
174         // Oid for SPNego GSS-API mechanism.
175         Oid spnegoMechOid = new Oid("1.3.6.1.5.5.2");
176         // Oid for kerberos principal name
177         Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1");
178         // GSS name for server
179         GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid);
180         // GSS credentials for server
181         GSSCredential serverCreds = manager.createCredential(serverName,
182             GSSCredential.DEFAULT_LIFETIME,
183             new Oid[]{kerberosMechOid, spnegoMechOid},
184             GSSCredential.ACCEPT_ONLY);
185         // Create a GSS context
186         gssContext = manager.createContext(serverCreds);
187         // Get service ticket from the authorization header
188         String serviceTicketBase64 = getAuthHeader(request);
189         byte[] inToken = Base64.decode(serviceTicketBase64);
190         byte[] res = gssContext.acceptSecContext(inToken, 0, inToken.length);
191         if(res != null) {
192           outToken = Base64.encodeBytes(res).replace("\n", "");
193         }
194         // Authenticate or deny based on its context completion
195         if (!gssContext.isEstablished()) {
196           throw new HttpAuthenticationException("Kerberos authentication failed: " +
197               "unable to establish context with the service ticket " +
198               "provided by the client.");
199         }
200         return SecurityUtil.getUserFromPrincipal(gssContext.getSrcName().toString());
201       } catch (GSSException e) {
202         throw new HttpAuthenticationException("Kerberos authentication failed: ", e);
203       } finally {
204         if (gssContext != null) {
205           try {
206             gssContext.dispose();
207           } catch (GSSException e) {
208             LOG.warn("Error while disposing GSS Context", e);
209           }
210         }
211       }
212     }
213 
214     /**
215      * Returns the base64 encoded auth header payload
216      *
217      * @throws HttpAuthenticationException if a remote or network exception occurs
218      */
219     private String getAuthHeader(HttpServletRequest request)
220         throws HttpAuthenticationException {
221       String authHeader = request.getHeader(AUTHORIZATION);
222       // Each http request must have an Authorization header
223       if (authHeader == null || authHeader.isEmpty()) {
224         throw new HttpAuthenticationException("Authorization header received " +
225             "from the client is empty.");
226       }
227       String authHeaderBase64String;
228       int beginIndex = (NEGOTIATE + " ").length();
229       authHeaderBase64String = authHeader.substring(beginIndex);
230       // Authorization header must have a payload
231       if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) {
232         throw new HttpAuthenticationException("Authorization header received " +
233             "from the client does not contain any data.");
234       }
235       return authHeaderBase64String;
236     }
237   }
238 }