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 019package org.apache.hadoop.hbase.thrift; 020 021import java.io.IOException; 022import java.security.PrivilegedExceptionAction; 023import java.util.Base64; 024 025import javax.servlet.ServletException; 026import javax.servlet.http.HttpServletRequest; 027import javax.servlet.http.HttpServletResponse; 028 029import org.apache.hadoop.conf.Configuration; 030import org.apache.hadoop.hbase.security.SecurityUtil; 031import org.apache.hadoop.security.UserGroupInformation; 032import org.apache.hadoop.security.authorize.AuthorizationException; 033import org.apache.hadoop.security.authorize.ProxyUsers; 034import org.apache.thrift.TProcessor; 035import org.apache.thrift.protocol.TProtocolFactory; 036import org.apache.thrift.server.TServlet; 037import org.apache.yetus.audience.InterfaceAudience; 038import org.ietf.jgss.GSSContext; 039import org.ietf.jgss.GSSCredential; 040import org.ietf.jgss.GSSException; 041import org.ietf.jgss.GSSManager; 042import org.ietf.jgss.GSSName; 043import org.ietf.jgss.Oid; 044import org.slf4j.Logger; 045import org.slf4j.LoggerFactory; 046 047/** 048 * Thrift Http Servlet is used for performing Kerberos authentication if security is enabled and 049 * also used for setting the user specified in "doAs" parameter. 050 */ 051@InterfaceAudience.Private 052public class ThriftHttpServlet extends TServlet { 053 private static final long serialVersionUID = 1L; 054 private static final Logger LOG = LoggerFactory.getLogger(ThriftHttpServlet.class.getName()); 055 private transient final UserGroupInformation realUser; 056 private transient final Configuration conf; 057 private final boolean securityEnabled; 058 private final boolean doAsEnabled; 059 private transient ThriftServerRunner.HBaseHandler hbaseHandler; 060 061 // HTTP Header related constants. 062 public static final String WWW_AUTHENTICATE = "WWW-Authenticate"; 063 public static final String AUTHORIZATION = "Authorization"; 064 public static final String NEGOTIATE = "Negotiate"; 065 066 public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory, 067 UserGroupInformation realUser, Configuration conf, ThriftServerRunner.HBaseHandler 068 hbaseHandler, boolean securityEnabled, boolean doAsEnabled) { 069 super(processor, protocolFactory); 070 this.realUser = realUser; 071 this.conf = conf; 072 this.hbaseHandler = hbaseHandler; 073 this.securityEnabled = securityEnabled; 074 this.doAsEnabled = doAsEnabled; 075 } 076 077 @Override 078 protected void doPost(HttpServletRequest request, HttpServletResponse response) 079 throws ServletException, IOException { 080 String effectiveUser = request.getRemoteUser(); 081 if (securityEnabled) { 082 try { 083 // As Thrift HTTP transport doesn't support SPNEGO yet (THRIFT-889), 084 // Kerberos authentication is being done at servlet level. 085 final RemoteUserIdentity identity = doKerberosAuth(request); 086 effectiveUser = identity.principal; 087 // It is standard for client applications expect this header. 088 // Please see http://tools.ietf.org/html/rfc4559 for more details. 089 response.addHeader(WWW_AUTHENTICATE, NEGOTIATE + " " + identity.outToken); 090 } catch (HttpAuthenticationException e) { 091 LOG.error("Kerberos Authentication failed", e); 092 // Send a 401 to the client 093 response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); 094 response.addHeader(WWW_AUTHENTICATE, NEGOTIATE); 095 response.getWriter().println("Authentication Error: " + e.getMessage()); 096 return; 097 } 098 } 099 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.getDecoder().decode(serviceTicketBase64); 190 byte[] res = gssContext.acceptSecContext(inToken, 0, inToken.length); 191 if(res != null) { 192 outToken = Base64.getEncoder().encodeToString(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}