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.ipc;
20  
21  import java.io.BufferedInputStream;
22  import java.io.BufferedOutputStream;
23  import java.io.DataInputStream;
24  import java.io.DataOutputStream;
25  import java.io.IOException;
26  import java.io.InputStream;
27  import java.io.OutputStream;
28  import java.net.InetSocketAddress;
29  import java.net.SocketTimeoutException;
30  import java.security.PrivilegedExceptionAction;
31  import java.util.HashMap;
32  import java.util.Map;
33  import java.util.Random;
34  
35  import javax.net.SocketFactory;
36  
37  import org.apache.commons.logging.Log;
38  import org.apache.commons.logging.LogFactory;
39  import org.apache.hadoop.conf.Configuration;
40  import org.apache.hadoop.hbase.security.HBaseMultiRealmUserAuthentication;
41  import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
42  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.AuthMethod;
43  import org.apache.hadoop.hbase.security.KerberosInfo;
44  import org.apache.hadoop.hbase.security.TokenInfo;
45  import org.apache.hadoop.hbase.security.User;
46  import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
47  import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector;
48  import org.apache.hadoop.io.DataOutputBuffer;
49  import org.apache.hadoop.io.IOUtils;
50  import org.apache.hadoop.io.Text;
51  import org.apache.hadoop.io.Writable;
52  import org.apache.hadoop.io.WritableUtils;
53  import org.apache.hadoop.ipc.RemoteException;
54  import org.apache.hadoop.net.NetUtils;
55  import org.apache.hadoop.security.SecurityUtil;
56  import org.apache.hadoop.security.UserGroupInformation;
57  import org.apache.hadoop.security.token.Token;
58  import org.apache.hadoop.security.token.TokenIdentifier;
59  import org.apache.hadoop.security.token.TokenSelector;
60  import org.apache.hadoop.util.ReflectionUtils;
61  
62  import javax.security.sasl.SaslException;
63  
64  /**
65   * A client for an IPC service, which support SASL authentication of connections
66   * using either GSSAPI for Kerberos authentication or DIGEST-MD5 for
67   * authentication using signed tokens.
68   *
69   * <p>
70   * This is a copy of org.apache.hadoop.ipc.Client from secure Hadoop,
71   * reworked to remove code duplicated with
72   * {@link org.apache.hadoop.hbase.HBaseClient}.  This is part of the loadable
73   * {@link SecureRpcEngine}, and only functions in connection with a
74   * {@link SecureServer} instance.
75   * </p>
76   */
77  public class SecureClient extends HBaseClient {
78  
79    private static final Log LOG =
80      LogFactory.getLog("org.apache.hadoop.ipc.SecureClient");
81  
82    public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY =
83        "hbase.ipc.client.fallback-to-simple-auth-allowed";
84    public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
85  
86    protected static Map<String,TokenSelector<? extends TokenIdentifier>> tokenHandlers =
87        new HashMap<String,TokenSelector<? extends TokenIdentifier>>();
88    static {
89      tokenHandlers.put(AuthenticationTokenIdentifier.AUTH_TOKEN_TYPE.toString(),
90          new AuthenticationTokenSelector());
91    }
92  
93    /** Thread that reads responses and notifies callers.  Each connection owns a
94     * socket connected to a remote address.  Calls are multiplexed through this
95     * socket: responses may be delivered out of order. */
96    protected class SecureConnection extends Connection {
97      private InetSocketAddress server;             // server ip:port
98      private String serverPrincipal;  // server's krb5 principal name
99      private SecureConnectionHeader header;              // connection header
100     private AuthMethod authMethod; // authentication method
101     private boolean useSasl;
102     private Token<? extends TokenIdentifier> token;
103     private HBaseSaslRpcClient saslRpcClient;
104     private int reloginMaxBackoff; // max pause before relogin on sasl failure
105 
106     public SecureConnection(ConnectionId remoteId) throws IOException {
107       super(remoteId);
108       this.server = remoteId.getAddress();
109 
110       User ticket = remoteId.getTicket();
111       Class<?> protocol = remoteId.getProtocol();
112       this.useSasl = User.isHBaseSecurityEnabled(conf);
113       if (useSasl && protocol != null) {
114         TokenInfo tokenInfo = protocol.getAnnotation(TokenInfo.class);
115         if (tokenInfo != null) {
116           TokenSelector<? extends TokenIdentifier> tokenSelector =
117               tokenHandlers.get(tokenInfo.value());
118           if (tokenSelector != null) {
119             token = tokenSelector.selectToken(new Text(clusterId),
120                 ticket.getUGI().getTokens());
121           } else if (LOG.isDebugEnabled()) {
122             LOG.debug("No token selector found for type "+tokenInfo.value());
123           }
124         }
125         KerberosInfo krbInfo = protocol.getAnnotation(KerberosInfo.class);
126         if (krbInfo != null) {
127           String serverKey = krbInfo.serverPrincipal();
128           if (serverKey == null) {
129             throw new IOException(
130                 "Can't obtain server Kerberos config key from KerberosInfo");
131           }
132           serverPrincipal = SecurityUtil.getServerPrincipal(
133               conf.get(serverKey), server.getAddress().getCanonicalHostName().toLowerCase());
134           if (LOG.isDebugEnabled()) {
135             LOG.debug("RPC Server Kerberos principal name for protocol="
136                 + protocol.getCanonicalName() + " is " + serverPrincipal);
137           }
138         }
139       }
140 
141       if (!useSasl) {
142         authMethod = AuthMethod.SIMPLE;
143       } else if (token != null) {
144         authMethod = AuthMethod.DIGEST;
145       } else if (HBaseMultiRealmUserAuthentication.isAUserInADifferentRealm(
146           ticket.getUGI(), conf)) {
147         authMethod = AuthMethod.KERBEROS_USER_REALM;
148         serverPrincipal =
149           HBaseMultiRealmUserAuthentication.replaceRealmWithUserRealm(serverPrincipal, conf);
150         if (LOG.isDebugEnabled()){
151           LOG.debug("AuthMehod is KERBEROS_USER_REALM and serverPrincipal is changed to "
152               + serverPrincipal);
153         }
154       } else {
155         authMethod = AuthMethod.KERBEROS;
156       }
157 
158       header = new SecureConnectionHeader(
159           protocol == null ? null : protocol.getName(), ticket, authMethod);
160 
161       if (LOG.isDebugEnabled())
162         LOG.debug("Use " + authMethod + " authentication for protocol "
163             + protocol.getSimpleName());
164 
165       reloginMaxBackoff = conf.getInt("hbase.security.relogin.maxbackoff", 5000);
166     }
167 
168     private synchronized void disposeSasl() {
169       if (saslRpcClient != null) {
170         try {
171           saslRpcClient.dispose();
172           saslRpcClient = null;
173         } catch (IOException ioe) {
174           LOG.info("Error disposing of SASL client", ioe);
175         }
176       }
177     }
178 
179     private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
180       UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
181       UserGroupInformation currentUser =
182         UserGroupInformation.getCurrentUser();
183       UserGroupInformation realUser = currentUser.getRealUser();
184       return (authMethod == AuthMethod.KERBEROS ||
185           authMethod == AuthMethod.KERBEROS_USER_REALM) &&
186           loginUser != null &&
187           //Make sure user logged in using Kerberos either keytab or TGT
188           loginUser.hasKerberosCredentials() &&
189           // relogin only in case it is the login user (e.g. JT)
190           // or superuser (like oozie).
191           (loginUser.equals(currentUser) || loginUser.equals(realUser));
192     }
193 
194     private synchronized boolean setupSaslConnection(final InputStream in2,
195         final OutputStream out2)
196         throws IOException {
197       saslRpcClient = new HBaseSaslRpcClient(authMethod, token, serverPrincipal, fallbackAllowed);
198       return saslRpcClient.saslConnect(in2, out2);
199     }
200 
201     /**
202      * If multiple clients with the same principal try to connect
203      * to the same server at the same time, the server assumes a
204      * replay attack is in progress. This is a feature of kerberos.
205      * In order to work around this, what is done is that the client
206      * backs off randomly and tries to initiate the connection
207      * again.
208      * The other problem is to do with ticket expiry. To handle that,
209      * a relogin is attempted.
210      * <p>
211      * The retry logic is governed by the {@link #shouldAuthenticateOverKrb}
212      * method. In case when the user doesn't have valid credentials, we don't
213      * need to retry (from cache or ticket). In such cases, it is prudent to
214      * throw a runtime exception when we receive a SaslException from the
215      * underlying authentication implementation, so there is no retry from 
216      * other high level (for eg, HCM or HBaseAdmin).
217      * </p>
218      */
219     private synchronized void handleSaslConnectionFailure(
220         final int currRetries,
221         final int maxRetries, final Exception ex, final Random rand,
222         final User user)
223     throws IOException, InterruptedException{
224       user.runAs(new PrivilegedExceptionAction<Object>() {
225         public Object run() throws IOException, InterruptedException {
226           closeConnection();
227           if (shouldAuthenticateOverKrb()) {
228             if (currRetries < maxRetries) {
229               LOG.debug("Exception encountered while connecting to " +
230                   "the server : " + ex);
231               //try re-login
232               if (UserGroupInformation.isLoginKeytabBased()) {
233                 UserGroupInformation.getLoginUser().reloginFromKeytab();
234               } else {
235                 UserGroupInformation.getLoginUser().reloginFromTicketCache();
236               }
237               disposeSasl();
238               //have granularity of milliseconds
239               //we are sleeping with the Connection lock held but since this
240               //connection instance is being used for connecting to the server
241               //in question, it is okay
242               Thread.sleep((rand.nextInt(reloginMaxBackoff) + 1));
243               return null;
244             } else {
245               String msg = "Couldn't setup connection for " +
246               UserGroupInformation.getLoginUser().getUserName() +
247               " to " + serverPrincipal;
248               LOG.warn(msg);
249               throw (IOException) new IOException(msg).initCause(ex);
250             }
251           } else {
252             LOG.warn("Exception encountered while connecting to " +
253                 "the server : " + ex);
254           }
255           if (ex instanceof RemoteException) {
256             throw (RemoteException)ex;
257           }
258           if (ex instanceof SaslException) {
259             String msg = "SASL authentication failed." +
260               " The most likely cause is missing or invalid credentials." +
261               " Consider 'kinit'.";
262             LOG.fatal(msg, ex);
263             throw new RuntimeException(msg, ex);
264           }
265           throw new IOException(ex);
266         }
267       });
268     }
269 
270     @Override
271     protected synchronized void setupIOstreams()
272         throws IOException, InterruptedException {
273       if (socket != null || shouldCloseConnection.get()) {
274         return;
275       }
276 
277       try {
278         if (LOG.isDebugEnabled()) {
279           LOG.debug("Connecting to "+server);
280         }
281         short numRetries = 0;
282         final short MAX_RETRIES = 5;
283         Random rand = null;
284         while (true) {
285           setupConnection();
286           InputStream inStream = NetUtils.getInputStream(socket);
287           OutputStream outStream = NetUtils.getOutputStream(socket);
288           writeRpcHeader(outStream);
289           if (useSasl) {
290             final InputStream in2 = inStream;
291             final OutputStream out2 = outStream;
292             User ticket = remoteId.getTicket();
293             if (authMethod == AuthMethod.KERBEROS ||
294 		authMethod == AuthMethod.KERBEROS_USER_REALM) {
295               UserGroupInformation ugi = ticket.getUGI();
296               if (ugi != null && ugi.getRealUser() != null) {
297                 ticket = User.create(ugi.getRealUser());
298               }
299             }
300             boolean continueSasl = false;
301             try {
302               continueSasl =
303                 ticket.runAs(new PrivilegedExceptionAction<Boolean>() {
304                   @Override
305                   public Boolean run() throws IOException {
306                     return setupSaslConnection(in2, out2);
307                   }
308                 });
309             } catch (Exception ex) {
310               if (rand == null) {
311                 rand = new Random();
312               }
313               handleSaslConnectionFailure(numRetries++, MAX_RETRIES, ex, rand,
314                    ticket);
315               continue;
316             }
317             if (continueSasl) {
318               // Sasl connect is successful. Let's set up Sasl i/o streams.
319               inStream = saslRpcClient.getInputStream(inStream);
320               outStream = saslRpcClient.getOutputStream(outStream);
321             } else {
322               // fall back to simple auth because server told us so.
323               authMethod = AuthMethod.SIMPLE;
324               header = new SecureConnectionHeader(header.getProtocol(),
325                   header.getUser(), authMethod);
326               useSasl = false;
327             }
328           }
329           this.in = new DataInputStream(new BufferedInputStream
330               (new PingInputStream(inStream)));
331           this.out = new DataOutputStream
332           (new BufferedOutputStream(outStream));
333           writeHeader();
334 
335           // update last activity time
336           touch();
337 
338           // start the receiver thread after the socket connection has been set up
339           start();
340           return;
341         }
342       } catch (IOException e) {
343         markClosed(e);
344         close();
345 
346         throw e;
347       }
348     }
349 
350     /* Write the RPC header */
351     private void writeRpcHeader(OutputStream outStream) throws IOException {
352       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(outStream));
353       // Write out the header, version and authentication method
354       out.write(SecureServer.HEADER.array());
355       out.write(SecureServer.CURRENT_VERSION);
356       authMethod.write(out);
357       out.flush();
358     }
359 
360     /**
361      * Write the protocol header for each connection
362      * Out is not synchronized because only the first thread does this.
363      */
364     private void writeHeader() throws IOException {
365       // Write out the ConnectionHeader
366       DataOutputBuffer buf = new DataOutputBuffer();
367       header.write(buf);
368 
369       // Write out the payload length
370       int bufLen = buf.getLength();
371       out.writeInt(bufLen);
372       out.write(buf.getData(), 0, bufLen);
373     }
374 
375     @Override
376     protected void receiveResponse() {
377       if (shouldCloseConnection.get()) {
378         return;
379       }
380       touch();
381 
382       try {
383         int id = in.readInt();                    // try to read an id
384 
385         if (LOG.isDebugEnabled())
386           LOG.debug(getName() + " got value #" + id);
387 
388         Call call = calls.remove(id);
389 
390         int state = in.readInt();     // read call status
391         if (LOG.isDebugEnabled()) {
392           LOG.debug("call #"+id+" state is " + state);
393         }
394         if (state == Status.SUCCESS.state) {
395           Writable value = ReflectionUtils.newInstance(valueClass, conf);
396           value.readFields(in);                 // read value
397           if (LOG.isDebugEnabled()) {
398             LOG.debug("call #"+id+", response is:\n"+value.toString());
399           }
400           // it's possible that this call may have been cleaned up due to a RPC
401           // timeout, so check if it still exists before setting the value.
402           if (call != null) {
403             call.setValue(value);
404           }
405         } else if (state == Status.ERROR.state) {
406           if (call != null) {
407             call.setException(new RemoteException(WritableUtils.readString(in), WritableUtils
408                 .readString(in)));
409           }
410         } else if (state == Status.FATAL.state) {
411           // Close the connection
412           markClosed(new RemoteException(WritableUtils.readString(in),
413                                          WritableUtils.readString(in)));
414         }
415       } catch (IOException e) {
416         if (e instanceof SocketTimeoutException && remoteId.rpcTimeout > 0) {
417           // Clean up open calls but don't treat this as a fatal condition,
418           // since we expect certain responses to not make it by the specified
419           // {@link ConnectionId#rpcTimeout}.
420           closeException = e;
421         } else {
422           // Since the server did not respond within the default ping interval
423           // time, treat this as a fatal condition and close this connection
424           markClosed(e);
425         }
426       } finally {
427         if (remoteId.rpcTimeout > 0) {
428           cleanupCalls(remoteId.rpcTimeout);
429         }
430       }
431     }
432 
433     /** Close the connection. */
434     protected synchronized void close() {
435       if (!shouldCloseConnection.get()) {
436         LOG.error("The connection is not in the closed state");
437         return;
438       }
439 
440       // release the resources
441       // first thing to do;take the connection out of the connection list
442       synchronized (connections) {
443         connections.remove(remoteId, this);
444       }
445 
446       // close the streams and therefore the socket
447       IOUtils.closeStream(out);
448       IOUtils.closeStream(in);
449       disposeSasl();
450 
451       // clean up all calls
452       if (closeException == null) {
453         if (!calls.isEmpty()) {
454           LOG.warn(
455               "A connection is closed for no cause and calls are not empty");
456 
457           // clean up calls anyway
458           closeException = new IOException("Unexpected closed connection");
459           cleanupCalls();
460         }
461       } else {
462         // log the info
463         if (LOG.isDebugEnabled()) {
464           LOG.debug("closing ipc connection to " + server + ": " +
465               closeException.getMessage(),closeException);
466         }
467 
468         // cleanup calls
469         cleanupCalls();
470       }
471       if (LOG.isDebugEnabled())
472         LOG.debug(getName() + ": closed");
473     }
474   }
475 
476   private final boolean fallbackAllowed;
477 
478   /**
479    * Construct an IPC client whose values are of the given {@link org.apache.hadoop.io.Writable}
480    * class.
481    * @param valueClass value class
482    * @param conf configuration
483    * @param factory socket factory
484    */
485   public SecureClient(Class<? extends Writable> valueClass, Configuration conf,
486       SocketFactory factory) {
487     super(valueClass, conf, factory);
488     this.fallbackAllowed =
489       conf.getBoolean(IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY,
490         IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT);
491     if (LOG.isDebugEnabled()) {
492       LOG.debug("fallbackAllowed=" + this.fallbackAllowed);
493     }
494   }
495 
496   /**
497    * Construct an IPC client with the default SocketFactory
498    * @param valueClass value class
499    * @param conf configuration
500    */
501   public SecureClient(Class<? extends Writable> valueClass, Configuration conf) {
502     this(valueClass, conf, NetUtils.getDefaultSocketFactory(conf));
503   }
504 
505   /**
506    * Creates a SecureConnection. Can be overridden by a subclass for testing.
507    * @param remoteId - the ConnectionId to use for the connection creation.
508    */
509   @Override
510   protected SecureConnection createConnection(ConnectionId remoteId) throws IOException {
511     return new SecureConnection(remoteId);
512   }
513 }