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 static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION;
22  
23  import java.io.ByteArrayInputStream;
24  import java.io.DataInputStream;
25  import java.io.DataOutputStream;
26  import java.io.IOException;
27  import java.net.InetAddress;
28  import java.nio.ByteBuffer;
29  import java.nio.channels.Channels;
30  import java.nio.channels.ReadableByteChannel;
31  import java.nio.channels.SocketChannel;
32  import java.security.PrivilegedExceptionAction;
33  import java.util.LinkedList;
34  import java.util.Set;
35  
36  import javax.security.sasl.Sasl;
37  import javax.security.sasl.SaslException;
38  import javax.security.sasl.SaslServer;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.HConstants;
44  import org.apache.hadoop.hbase.io.HbaseObjectWritable;
45  import org.apache.hadoop.hbase.io.WritableWithSize;
46  import org.apache.hadoop.hbase.security.HBaseMultiRealmUserAuthentication;
47  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer;
48  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.AuthMethod;
49  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslDigestCallbackHandler;
50  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslGssCallbackHandler;
51  import org.apache.hadoop.hbase.security.HBaseSaslRpcServer.SaslStatus;
52  import org.apache.hadoop.hbase.security.User;
53  import org.apache.hadoop.hbase.util.ByteBufferOutputStream;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.io.BytesWritable;
56  import org.apache.hadoop.io.IntWritable;
57  import org.apache.hadoop.io.Writable;
58  import org.apache.hadoop.io.WritableUtils;
59  import org.apache.hadoop.security.AccessControlException;
60  import org.apache.hadoop.security.UserGroupInformation;
61  import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
62  import org.apache.hadoop.security.authorize.AuthorizationException;
63  import org.apache.hadoop.security.authorize.ProxyUsers;
64  import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
65  import org.apache.hadoop.security.token.SecretManager;
66  import org.apache.hadoop.security.token.SecretManager.InvalidToken;
67  import org.apache.hadoop.security.token.TokenIdentifier;
68  import org.apache.hadoop.util.ReflectionUtils;
69  import org.apache.hadoop.util.StringUtils;
70  
71  import com.google.common.collect.ImmutableSet;
72  
73  /**
74   * An abstract IPC service, supporting SASL authentication of connections,
75   * using GSSAPI for Kerberos authentication or DIGEST-MD5 for authentication
76   * via signed tokens.
77   *
78   * <p>
79   * This is part of the {@link SecureRpcEngine} implementation.
80   * </p>
81   *
82   * @see org.apache.hadoop.hbase.ipc.SecureClient
83   */
84  public abstract class SecureServer extends HBaseServer {
85    private final boolean authorize;
86    private boolean isSecurityEnabled;
87  
88    /**
89     * The first four bytes of secure RPC connections
90     */
91    public static final ByteBuffer HEADER = ByteBuffer.wrap("srpc".getBytes());
92  
93    // 1 : Introduce ping and server does not throw away RPCs
94    // 3 : Introduce the protocol into the RPC connection header
95    // 4 : Introduced SASL security layer
96    public static final byte CURRENT_VERSION = 4;
97    public static final Set<Byte> INSECURE_VERSIONS = ImmutableSet.of((byte) 3);
98  
99    public static final Log LOG = LogFactory.getLog(SecureServer.class);
100   private static final Log AUDITLOG = LogFactory.getLog("SecurityLogger." +
101     SecureServer.class.getName());
102 
103   private static final String AUTH_FAILED_FOR = "Auth failed for ";
104   private static final String AUTH_SUCCESSFUL_FOR = "Auth successful for ";
105 
106   protected SecretManager<TokenIdentifier> secretManager;
107   protected ServiceAuthorizationManager authManager;
108 
109   protected class SecureCall extends HBaseServer.Call {
110     public SecureCall(int id, Writable param, Connection connection,
111         Responder responder, long size) {
112       super(id, param, connection, responder, size);
113     }
114 
115     @Override
116     protected synchronized void setResponse(Object value, Status status,
117         String errorClass, String error) {
118       Writable result = null;
119       if (value instanceof Writable) {
120         result = (Writable) value;
121       } else {
122         /* We might have a null value and errors. Avoid creating a
123          * HbaseObjectWritable, because the constructor fails on null. */
124         if (value != null) {
125           result = new HbaseObjectWritable(value);
126         }
127       }
128 
129       int size = BUFFER_INITIAL_SIZE;
130       if (result instanceof WritableWithSize) {
131         // get the size hint.
132         WritableWithSize ohint = (WritableWithSize) result;
133         long hint = ohint.getWritableSize() + Bytes.SIZEOF_INT + Bytes.SIZEOF_INT;
134         if (hint > Integer.MAX_VALUE) {
135           // oops, new problem.
136           IOException ioe =
137             new IOException("Result buffer size too large: " + hint);
138           errorClass = ioe.getClass().getName();
139           error = StringUtils.stringifyException(ioe);
140         } else {
141           size = (int)hint;
142         }
143       }
144 
145       ByteBufferOutputStream buf = new ByteBufferOutputStream(size);
146       DataOutputStream out = new DataOutputStream(buf);
147       try {
148         out.writeInt(this.id);                // write call id
149         out.writeInt(status.state);           // write status
150       } catch (IOException e) {
151         errorClass = e.getClass().getName();
152         error = StringUtils.stringifyException(e);
153       }
154 
155       try {
156         if (status == Status.SUCCESS) {
157           result.write(out);
158         } else {
159           WritableUtils.writeString(out, errorClass);
160           WritableUtils.writeString(out, error);
161         }
162         if (((SecureConnection)connection).useWrap) {
163           wrapWithSasl(buf);
164         }
165       } catch (IOException e) {
166         LOG.warn("Error sending response to call: ", e);
167       }
168 
169       this.response = buf.getByteBuffer();
170     }
171 
172     private void wrapWithSasl(ByteBufferOutputStream response)
173         throws IOException {
174       if (((SecureConnection)connection).useSasl) {
175         // getByteBuffer calls flip()
176         ByteBuffer buf = response.getByteBuffer();
177         byte[] token;
178         // synchronization may be needed since there can be multiple Handler
179         // threads using saslServer to wrap responses.
180         synchronized (((SecureConnection)connection).saslServer) {
181           token = ((SecureConnection)connection).saslServer.wrap(buf.array(),
182               buf.arrayOffset(), buf.remaining());
183         }
184         if (LOG.isTraceEnabled()) {
185           LOG.trace("Adding saslServer wrapped token of size " + token.length
186               + " as call response.");
187         }
188         buf.clear();
189         DataOutputStream saslOut = new DataOutputStream(response);
190         saslOut.writeInt(token.length);
191         saslOut.write(token, 0, token.length);
192       }
193     }
194   }
195 
196   /** Reads calls from a connection and queues them for handling. */
197   public class SecureConnection extends HBaseServer.Connection  {
198     private boolean rpcHeaderRead = false; // if initial rpc header is read
199     private boolean headerRead = false;  //if the connection header that
200                                          //follows version is read.
201     private ByteBuffer data;
202     private ByteBuffer dataLengthBuffer;
203     protected final LinkedList<SecureCall> responseQueue;
204     private int dataLength;
205     private InetAddress addr;
206 
207     boolean useSasl;
208     SaslServer saslServer;
209     private AuthMethod authMethod;
210     private boolean saslContextEstablished;
211     private boolean skipInitialSaslHandshake;
212     private ByteBuffer rpcHeaderBuffer;
213     private ByteBuffer unwrappedData;
214     private ByteBuffer unwrappedDataLengthBuffer;
215 
216     public UserGroupInformation attemptingUser = null; // user name before auth
217 
218     // Fake 'call' for failed authorization response
219     private final int AUTHORIZATION_FAILED_CALLID = -1;
220     // Fake 'call' for SASL context setup
221     private static final int SASL_CALLID = -33;
222     private final SecureCall saslCall = new SecureCall(SASL_CALLID, null, this, null, 0);
223 
224     private boolean useWrap = false;
225 
226     public SecureConnection(SocketChannel channel, long lastContact) {
227       super(channel, lastContact);
228       this.header = new SecureConnectionHeader();
229       this.channel = channel;
230       this.data = null;
231       this.dataLengthBuffer = ByteBuffer.allocate(4);
232       this.unwrappedData = null;
233       this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
234       this.socket = channel.socket();
235       this.addr = socket.getInetAddress();
236       this.responseQueue = new LinkedList<SecureCall>();
237     }
238 
239     @Override
240     public String toString() {
241       return getHostAddress() + ":" + remotePort;
242     }
243 
244     public String getHostAddress() {
245       return hostAddress;
246     }
247 
248     public InetAddress getHostInetAddress() {
249       return addr;
250     }
251 
252     private User getAuthorizedUgi(String authorizedId)
253         throws IOException {
254       if (authMethod == AuthMethod.DIGEST) {
255         TokenIdentifier tokenId = HBaseSaslRpcServer.getIdentifier(authorizedId,
256             secretManager);
257         UserGroupInformation ugi = tokenId.getUser();
258         if (ugi == null) {
259           throw new AccessControlException(
260               "Can't retrieve username from tokenIdentifier.");
261         }
262         ugi.addTokenIdentifier(tokenId);
263         return User.create(ugi);
264       } else {
265         return User.create(UserGroupInformation.createRemoteUser(authorizedId));
266       }
267     }
268 
269     private void saslReadAndProcess(byte[] saslToken) throws IOException,
270         InterruptedException {
271       if (!saslContextEstablished) {
272         byte[] replyToken = null;
273         try {
274           if (saslServer == null) {
275             switch (authMethod) {
276             case DIGEST:
277               if (secretManager == null) {
278                 throw new AccessControlException(
279                     "Server is not configured to do DIGEST authentication.");
280               }
281               saslServer = Sasl.createSaslServer(AuthMethod.DIGEST
282                   .getMechanismName(), null, HBaseSaslRpcServer.SASL_DEFAULT_REALM,
283                   HBaseSaslRpcServer.SASL_PROPS, new SaslDigestCallbackHandler(
284                       secretManager, this));
285               break;
286             case KERBEROS_USER_REALM:
287               UserGroupInformation ugi = HBaseMultiRealmUserAuthentication.
288                                                 getServerUGIForUserRealm(conf);
289               createSaslServer(ugi);
290               break;
291             default:
292               UserGroupInformation current = UserGroupInformation.getCurrentUser();
293               createSaslServer(current);
294             }
295             if (saslServer == null)
296               throw new AccessControlException(
297                   "Unable to find SASL server implementation for "
298                       + authMethod.getMechanismName());
299             if (LOG.isTraceEnabled()) {
300               LOG.trace("Created SASL server with mechanism = "
301                   + authMethod.getMechanismName());
302             }
303           }
304           if (LOG.isTraceEnabled()) {
305             LOG.trace("Have read input token of size " + saslToken.length
306                 + " for processing by saslServer.evaluateResponse()");
307           }
308           replyToken = saslServer.evaluateResponse(saslToken);
309         } catch (IOException e) {
310           IOException sendToClient = e;
311           Throwable cause = e;
312           while (cause != null) {
313             if (cause instanceof InvalidToken) {
314               sendToClient = (InvalidToken) cause;
315               break;
316             }
317             cause = cause.getCause();
318           }
319           doSaslReply(SaslStatus.ERROR, null, sendToClient.getClass().getName(),
320               sendToClient.getLocalizedMessage());
321           rpcMetrics.authenticationFailures.inc();
322           String clientIP = this.toString();
323           // attempting user could be null
324           AUDITLOG.warn(AUTH_FAILED_FOR + clientIP + ":" + attemptingUser);
325           throw e;
326         }
327         if (replyToken != null) {
328           if (LOG.isTraceEnabled()) {
329             LOG.trace("Will send token of size " + replyToken.length
330                 + " from saslServer.");
331           }
332           doSaslReply(SaslStatus.SUCCESS, new BytesWritable(replyToken), null,
333               null);
334         }
335         if (saslServer.isComplete()) {
336           if (LOG.isDebugEnabled()) {
337             LOG.debug("SASL server context established. Negotiated QoP is "
338               + saslServer.getNegotiatedProperty(Sasl.QOP));
339           }
340           String qop = (String) saslServer.getNegotiatedProperty(Sasl.QOP);
341           useWrap = qop != null && !"auth".equalsIgnoreCase(qop);
342           ticket = getAuthorizedUgi(saslServer.getAuthorizationID());
343           if (LOG.isDebugEnabled()) {
344             LOG.debug("SASL server successfully authenticated client: " + ticket);
345           }
346           rpcMetrics.authenticationSuccesses.inc();
347           AUDITLOG.info(AUTH_SUCCESSFUL_FOR + ticket);
348           saslContextEstablished = true;
349         }
350       } else {
351         if (LOG.isTraceEnabled()) {
352           LOG.trace("Have read input token of size " + saslToken.length
353               + " for processing by saslServer.unwrap()");
354         }
355         if (!useWrap) {
356           processOneRpc(saslToken);
357         } else {
358           byte[] plaintextData = saslServer.unwrap(saslToken, 0,
359               saslToken.length);
360           processUnwrappedData(plaintextData);
361         }
362       }
363     }
364 
365     private void createSaslServer(UserGroupInformation ugi)
366     throws AccessControlException, IOException, InterruptedException {
367       String fullName = ugi.getUserName();
368       if (LOG.isDebugEnabled())
369         LOG.debug("Kerberos principal name is " + fullName);
370       final String names[] = HBaseSaslRpcServer.splitKerberosName(fullName);
371       if (names.length != 3) {
372         throw new AccessControlException(
373             "Kerberos principal name does NOT have the expected "
374             + "hostname part: " + fullName);
375       }
376       ugi.doAs(new PrivilegedExceptionAction<Object>() {
377         @Override
378         public Object run() throws SaslException {
379           saslServer = Sasl.createSaslServer(AuthMethod.KERBEROS
380               .getMechanismName(), names[0], names[1],
381               HBaseSaslRpcServer.SASL_PROPS, new SaslGssCallbackHandler());
382           return null;
383         }
384       });
385     }
386 
387     private void doSaslReply(SaslStatus status, Writable rv,
388         String errorClass, String error) throws IOException {
389       saslCall.setResponse(rv,
390           status == SaslStatus.SUCCESS ? Status.SUCCESS : Status.ERROR,
391            errorClass, error);
392       saslCall.responder = responder;
393       saslCall.sendResponseIfReady();
394     }
395 
396     private void disposeSasl() {
397       if (saslServer != null) {
398         try {
399           saslServer.dispose();
400         } catch (SaslException ignored) {
401         }
402       }
403     }
404 
405     public int readAndProcess() throws IOException, InterruptedException {
406       while (true) {
407         /* Read at most one RPC. If the header is not read completely yet
408          * then iterate until we read first RPC or until there is no data left.
409          */
410         int count = -1;
411         if (dataLengthBuffer.remaining() > 0) {
412           count = channelRead(channel, dataLengthBuffer);
413           if (count < 0 || dataLengthBuffer.remaining() > 0)
414             return count;
415         }
416 
417         if (!rpcHeaderRead) {
418           //Every connection is expected to send the header.
419           if (rpcHeaderBuffer == null) {
420             rpcHeaderBuffer = ByteBuffer.allocate(2);
421           }
422           count = channelRead(channel, rpcHeaderBuffer);
423           if (count < 0 || rpcHeaderBuffer.remaining() > 0) {
424             return count;
425           }
426           int version = rpcHeaderBuffer.get(0);
427           byte[] method = new byte[] {rpcHeaderBuffer.get(1)};
428           authMethod = AuthMethod.read(new DataInputStream(
429               new ByteArrayInputStream(method)));
430           dataLengthBuffer.flip();
431           if (!HEADER.equals(dataLengthBuffer) || version != CURRENT_VERSION) {
432             //Warning is ok since this is not supposed to happen.
433             if (INSECURE_VERSIONS.contains(version)) {
434               LOG.warn("An insecure client (version '" + version + "') is attempting to connect " +
435                   " to this version '" + CURRENT_VERSION + "' secure server from " +
436                   hostAddress + ":" + remotePort);
437             } else {
438               LOG.warn("Incorrect header or version mismatch from " +
439                   hostAddress + ":" + remotePort +
440                   " got version " + version +
441                   " expected version " + CURRENT_VERSION);              
442             }
443             
444             return -1;
445           }
446           dataLengthBuffer.clear();
447           if (authMethod == null) {
448             throw new IOException("Unable to read authentication method");
449           }
450           if (isSecurityEnabled && authMethod == AuthMethod.SIMPLE) {
451             AccessControlException ae = new AccessControlException(
452                 "Authentication is required");
453             SecureCall failedCall = new SecureCall(AUTHORIZATION_FAILED_CALLID, null, this,
454                 null, 0);
455             failedCall.setResponse(null, Status.FATAL, ae.getClass().getName(),
456                 ae.getMessage());
457             responder.doRespond(failedCall);
458             throw ae;
459           }
460           if (!isSecurityEnabled && authMethod != AuthMethod.SIMPLE) {
461             doSaslReply(SaslStatus.SUCCESS, new IntWritable(
462                 HBaseSaslRpcServer.SWITCH_TO_SIMPLE_AUTH), null, null);
463             authMethod = AuthMethod.SIMPLE;
464             // client has already sent the initial Sasl message and we
465             // should ignore it. Both client and server should fall back
466             // to simple auth from now on.
467             skipInitialSaslHandshake = true;
468           }
469           if (authMethod != AuthMethod.SIMPLE) {
470             useSasl = true;
471           }
472 
473           rpcHeaderBuffer = null;
474           rpcHeaderRead = true;
475           continue;
476         }
477 
478         if (data == null) {
479           dataLengthBuffer.flip();
480           dataLength = dataLengthBuffer.getInt();
481 
482           if (dataLength == HBaseClient.PING_CALL_ID) {
483             if(!useWrap) { //covers the !useSasl too
484               dataLengthBuffer.clear();
485               return 0;  //ping message
486             }
487           }
488           if (dataLength < 0) {
489             LOG.warn("Unexpected data length " + dataLength + "!! from " +
490                 getHostAddress());
491           }
492           data = ByteBuffer.allocate(dataLength);
493           incRpcCount();  // Increment the rpc count
494         }
495 
496         count = channelRead(channel, data);
497 
498         if (data.remaining() == 0) {
499           dataLengthBuffer.clear();
500           data.flip();
501           if (skipInitialSaslHandshake) {
502             data = null;
503             skipInitialSaslHandshake = false;
504             continue;
505           }
506           boolean isHeaderRead = headerRead;
507           if (useSasl) {
508             saslReadAndProcess(data.array());
509           } else {
510             processOneRpc(data.array());
511           }
512           data = null;
513           if (!isHeaderRead) {
514             continue;
515           }
516         }
517         return count;
518       }
519     }
520 
521     /// Reads the connection header following version
522     private void processHeader(byte[] buf) throws IOException {
523       DataInputStream in =
524         new DataInputStream(new ByteArrayInputStream(buf));
525       header.readFields(in);
526       try {
527         String protocolClassName = header.getProtocol();
528         if (protocolClassName != null) {
529           protocol = getProtocolClass(header.getProtocol(), conf);
530         }
531       } catch (ClassNotFoundException cnfe) {
532         throw new IOException("Unknown protocol: " + header.getProtocol());
533       }
534 
535       User protocolUser = header.getUser();
536       if (!useSasl) {
537         ticket = protocolUser;
538         if (ticket != null) {
539           ticket.getUGI().setAuthenticationMethod(AuthMethod.SIMPLE.authenticationMethod);
540         }
541       } else {
542         // user is authenticated
543         ticket.getUGI().setAuthenticationMethod(authMethod.authenticationMethod);
544         //Now we check if this is a proxy user case. If the protocol user is
545         //different from the 'user', it is a proxy user scenario. However,
546         //this is not allowed if user authenticated with DIGEST.
547         if ((protocolUser != null)
548             && (!protocolUser.getName().equals(ticket.getName()))) {
549           if (authMethod == AuthMethod.DIGEST) {
550             // Not allowed to doAs if token authentication is used
551             throw new AccessControlException("Authenticated user (" + ticket
552                 + ") doesn't match what the client claims to be ("
553                 + protocolUser + ")");
554           } else {
555             // Effective user can be different from authenticated user
556             // for simple auth or kerberos auth
557             // The user is the real user. Now we create a proxy user
558             UserGroupInformation realUser = ticket.getUGI();
559             ticket = User.create(
560                 UserGroupInformation.createProxyUser(protocolUser.getName(),
561                     realUser));
562             // Now the user is a proxy user, set Authentication method Proxy.
563             ticket.getUGI().setAuthenticationMethod(AuthenticationMethod.PROXY);
564           }
565         }
566       }
567     }
568 
569     private void processUnwrappedData(byte[] inBuf) throws IOException,
570         InterruptedException {
571       ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(
572           inBuf));
573       // Read all RPCs contained in the inBuf, even partial ones
574       while (true) {
575         int count = -1;
576         if (unwrappedDataLengthBuffer.remaining() > 0) {
577           count = channelRead(ch, unwrappedDataLengthBuffer);
578           if (count <= 0 || unwrappedDataLengthBuffer.remaining() > 0)
579             return;
580         }
581 
582         if (unwrappedData == null) {
583           unwrappedDataLengthBuffer.flip();
584           int unwrappedDataLength = unwrappedDataLengthBuffer.getInt();
585 
586           if (unwrappedDataLength == HBaseClient.PING_CALL_ID) {
587             if (LOG.isTraceEnabled()) {
588               LOG.trace("Received ping message");
589             }
590             unwrappedDataLengthBuffer.clear();
591             continue; // ping message
592           }
593           unwrappedData = ByteBuffer.allocate(unwrappedDataLength);
594         }
595 
596         count = channelRead(ch, unwrappedData);
597         if (count <= 0 || unwrappedData.remaining() > 0)
598           return;
599 
600         if (unwrappedData.remaining() == 0) {
601           unwrappedDataLengthBuffer.clear();
602           unwrappedData.flip();
603           processOneRpc(unwrappedData.array());
604           unwrappedData = null;
605         }
606       }
607     }
608 
609     private void processOneRpc(byte[] buf) throws IOException,
610         InterruptedException {
611       if (headerRead) {
612         processData(buf);
613       } else {
614         processHeader(buf);
615         headerRead = true;
616         if (!authorizeConnection()) {
617           throw new AccessControlException("Connection from " + this
618               + " for protocol " + header.getProtocol()
619               + " is unauthorized for user " + ticket);
620         }
621       }
622     }
623 
624     protected void processData(byte[] buf) throws  IOException, InterruptedException {
625       DataInputStream dis =
626         new DataInputStream(new ByteArrayInputStream(buf));
627       int id = dis.readInt();                    // try to read an id
628 
629       if (LOG.isTraceEnabled()) {
630         LOG.trace(" got #" + id);
631       }
632 
633       Writable param = ReflectionUtils.newInstance(paramClass, conf);           // read param
634       param.readFields(dis);
635 
636       SecureCall call = new SecureCall(id, param, this, responder, buf.length);
637 
638       if (priorityCallQueue != null && getQosLevel(param) > highPriorityLevel) {
639         priorityCallQueue.put(call);
640         updateCallQueueLenMetrics(priorityCallQueue);
641       } else if (replicationQueue != null && getQosLevel(param) == HConstants.REPLICATION_QOS) {
642         replicationQueue.put(call);
643         updateCallQueueLenMetrics(replicationQueue);
644       } else {
645         callQueue.put(call);              // queue the call; maybe blocked here
646         updateCallQueueLenMetrics(callQueue);
647       }
648     }
649 
650     private boolean authorizeConnection() throws IOException {
651       try {
652         // If auth method is DIGEST, the token was obtained by the
653         // real user for the effective user, therefore not required to
654         // authorize real user. doAs is allowed only for simple or kerberos
655         // authentication
656         if (ticket != null && ticket.getUGI().getRealUser() != null
657             && (authMethod != AuthMethod.DIGEST)) {
658           ProxyUsers.authorize(ticket.getUGI(), this.getHostAddress(), conf);
659         }
660         authorize(ticket, header, getHostInetAddress());
661         if (LOG.isDebugEnabled()) {
662           LOG.debug("Successfully authorized " + header);
663         }
664         rpcMetrics.authorizationSuccesses.inc();
665       } catch (AuthorizationException ae) {
666         if (LOG.isDebugEnabled()) {
667           LOG.debug("Connection authorization failed: "+ae.getMessage(), ae);
668         }
669         rpcMetrics.authorizationFailures.inc();
670         SecureCall failedCall = new SecureCall(AUTHORIZATION_FAILED_CALLID, null, this,
671             null, 0);
672         failedCall.setResponse(null, Status.FATAL, ae.getClass().getName(),
673             ae.getMessage());
674         responder.doRespond(failedCall);
675         return false;
676       }
677       return true;
678     }
679 
680     protected synchronized void close() {
681       disposeSasl();
682       data = null;
683       dataLengthBuffer = null;
684       if (!channel.isOpen())
685         return;
686       try {socket.shutdownOutput();} catch(Exception ignored) {} // FindBugs DE_MIGHT_IGNORE
687       if (channel.isOpen()) {
688         try {channel.close();} catch(Exception ignored) {}
689       }
690       try {socket.close();} catch(Exception ignored) {}
691     }
692   }
693 
694   /** Constructs a server listening on the named port and address.  Parameters passed must
695    * be of the named class.  The <code>handlerCount</handlerCount> determines
696    * the number of handler threads that will be used to process calls.
697    *
698    */
699   @SuppressWarnings("unchecked")
700   protected SecureServer(String bindAddress, int port,
701                   Class<? extends Writable> paramClass, int handlerCount,
702                   int priorityHandlerCount, Configuration conf, String serverName,
703                   int highPriorityLevel)
704     throws IOException {
705     super(bindAddress, port, paramClass, handlerCount, priorityHandlerCount,
706         conf, serverName, highPriorityLevel);
707     this.authorize =
708       conf.getBoolean(HADOOP_SECURITY_AUTHORIZATION, false);
709     this.isSecurityEnabled = User.isHBaseSecurityEnabled(this.conf);
710 
711     if (isSecurityEnabled) {
712       HBaseSaslRpcServer.init(conf);
713     }
714   }
715 
716   @Override
717   protected Connection getConnection(SocketChannel channel, long time) {
718     return new SecureConnection(channel, time);
719   }
720 
721   Configuration getConf() {
722     return conf;
723   }
724 
725   /** for unit testing only, should be called before server is started */
726   void disableSecurity() {
727     this.isSecurityEnabled = false;
728   }
729 
730   /** for unit testing only, should be called before server is started */
731   void enableSecurity() {
732     this.isSecurityEnabled = true;
733   }
734 
735   /** Stops the service.  No new calls will be handled after this is called. */
736   public synchronized void stop() {
737     super.stop();
738   }
739 
740   public SecretManager<? extends TokenIdentifier> getSecretManager() {
741     return this.secretManager;
742   }
743 
744   public void setSecretManager(SecretManager<? extends TokenIdentifier> secretManager) {
745     this.secretManager = (SecretManager<TokenIdentifier>) secretManager;    
746   }
747 
748   /**
749    * Authorize the incoming client connection.
750    *
751    * @param user client user
752    * @param connection incoming connection
753    * @param addr InetAddress of incoming connection
754    * @throws org.apache.hadoop.security.authorize.AuthorizationException when the client isn't authorized to talk the protocol
755    */
756   public void authorize(User user,
757                         ConnectionHeader connection,
758                         InetAddress addr
759                         ) throws AuthorizationException {
760     if (authorize) {
761       Class<?> protocol = null;
762       try {
763         protocol = getProtocolClass(connection.getProtocol(), getConf());
764       } catch (ClassNotFoundException cfne) {
765         throw new AuthorizationException("Unknown protocol: " +
766                                          connection.getProtocol());
767       }
768       authManager.authorize(user != null ? user.getUGI() : null,
769           protocol, getConf(), addr);
770     }
771   }
772 }