|
@@ -366,6 +366,24 @@ public abstract class Server {
|
|
|
return (call != null ) ? call.getHostInetAddress() : null;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Returns the SASL qop for the current call, if the current call is
|
|
|
+ * set, and the SASL negotiation is done. Otherwise return null. Note
|
|
|
+ * that CurCall is thread local object. So in fact, different handler
|
|
|
+ * threads will process different CurCall object.
|
|
|
+ *
|
|
|
+ * Also, only return for RPC calls, not supported for other protocols.
|
|
|
+ * @return the QOP of the current connection.
|
|
|
+ */
|
|
|
+ public static String getEstablishedQOP() {
|
|
|
+ Call call = CurCall.get();
|
|
|
+ if (call == null || !(call instanceof RpcCall)) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ RpcCall rpcCall = (RpcCall)call;
|
|
|
+ return rpcCall.connection.getEstablishedQOP();
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Returns the clientId from the current RPC request
|
|
|
*/
|
|
@@ -445,6 +463,10 @@ public abstract class Server {
|
|
|
// maintains the set of client connections and handles idle timeouts
|
|
|
private ConnectionManager connectionManager;
|
|
|
private Listener listener = null;
|
|
|
+ // Auxiliary listeners maintained as in a map, to allow
|
|
|
+ // arbitrary number of of auxiliary listeners. A map from
|
|
|
+ // the port to the listener binding to it.
|
|
|
+ private Map<Integer, Listener> auxiliaryListenerMap;
|
|
|
private Responder responder = null;
|
|
|
private Handler[] handlers = null;
|
|
|
|
|
@@ -1028,11 +1050,12 @@ public abstract class Server {
|
|
|
private Reader[] readers = null;
|
|
|
private int currentReader = 0;
|
|
|
private InetSocketAddress address; //the address we bind at
|
|
|
+ private int listenPort; //the port we bind at
|
|
|
private int backlogLength = conf.getInt(
|
|
|
CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_KEY,
|
|
|
CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_DEFAULT);
|
|
|
|
|
|
- public Listener() throws IOException {
|
|
|
+ Listener(int port) throws IOException {
|
|
|
address = new InetSocketAddress(bindAddress, port);
|
|
|
// Create a new server socket and set to non blocking mode
|
|
|
acceptChannel = ServerSocketChannel.open();
|
|
@@ -1040,7 +1063,10 @@ public abstract class Server {
|
|
|
|
|
|
// Bind the server socket to the local host and port
|
|
|
bind(acceptChannel.socket(), address, backlogLength, conf, portRangeConfig);
|
|
|
- port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
|
|
|
+ //Could be an ephemeral port
|
|
|
+ this.listenPort = acceptChannel.socket().getLocalPort();
|
|
|
+ Thread.currentThread().setName("Listener at " +
|
|
|
+ bindAddress + "/" + this.listenPort);
|
|
|
// create a selector;
|
|
|
selector= Selector.open();
|
|
|
readers = new Reader[readThreads];
|
|
@@ -1223,7 +1249,7 @@ public abstract class Server {
|
|
|
channel.socket().setKeepAlive(true);
|
|
|
|
|
|
Reader reader = getReader();
|
|
|
- Connection c = connectionManager.register(channel);
|
|
|
+ Connection c = connectionManager.register(channel, this.listenPort);
|
|
|
// If the connectionManager can't take it, close the connection.
|
|
|
if (c == null) {
|
|
|
if (channel.isOpen()) {
|
|
@@ -1643,6 +1669,7 @@ public abstract class Server {
|
|
|
private ByteBuffer unwrappedDataLengthBuffer;
|
|
|
private int serviceClass;
|
|
|
private boolean shouldClose = false;
|
|
|
+ private int ingressPort;
|
|
|
|
|
|
UserGroupInformation user = null;
|
|
|
public UserGroupInformation attemptingUser = null; // user name before auth
|
|
@@ -1654,7 +1681,8 @@ public abstract class Server {
|
|
|
private boolean sentNegotiate = false;
|
|
|
private boolean useWrap = false;
|
|
|
|
|
|
- public Connection(SocketChannel channel, long lastContact) {
|
|
|
+ public Connection(SocketChannel channel, long lastContact,
|
|
|
+ int ingressPort) {
|
|
|
this.channel = channel;
|
|
|
this.lastContact = lastContact;
|
|
|
this.data = null;
|
|
@@ -1666,6 +1694,7 @@ public abstract class Server {
|
|
|
this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
|
|
|
this.socket = channel.socket();
|
|
|
this.addr = socket.getInetAddress();
|
|
|
+ this.ingressPort = ingressPort;
|
|
|
if (addr == null) {
|
|
|
this.hostAddress = "*Unknown*";
|
|
|
} else {
|
|
@@ -1700,9 +1729,24 @@ public abstract class Server {
|
|
|
return hostAddress;
|
|
|
}
|
|
|
|
|
|
+ public int getIngressPort() {
|
|
|
+ return ingressPort;
|
|
|
+ }
|
|
|
+
|
|
|
public InetAddress getHostInetAddress() {
|
|
|
return addr;
|
|
|
}
|
|
|
+
|
|
|
+ public String getEstablishedQOP() {
|
|
|
+ // In practice, saslServer should not be null when this is
|
|
|
+ // called. If it is null, it must be either some
|
|
|
+ // configuration mistake or it is called from unit test.
|
|
|
+ if (saslServer == null) {
|
|
|
+ LOG.warn("SASL server should not be null!");
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ return (String)saslServer.getNegotiatedProperty(Sasl.QOP);
|
|
|
+ }
|
|
|
|
|
|
public void setLastContact(long lastContact) {
|
|
|
this.lastContact = lastContact;
|
|
@@ -2175,7 +2219,7 @@ public abstract class Server {
|
|
|
private SaslServer createSaslServer(AuthMethod authMethod)
|
|
|
throws IOException, InterruptedException {
|
|
|
final Map<String,?> saslProps =
|
|
|
- saslPropsResolver.getServerProperties(addr);
|
|
|
+ saslPropsResolver.getServerProperties(addr, ingressPort);
|
|
|
return new SaslRpcServer(authMethod).create(this, saslProps, secretManager);
|
|
|
}
|
|
|
|
|
@@ -2655,7 +2699,8 @@ public abstract class Server {
|
|
|
private class Handler extends Thread {
|
|
|
public Handler(int instanceNumber) {
|
|
|
this.setDaemon(true);
|
|
|
- this.setName("IPC Server handler "+ instanceNumber + " on " + port);
|
|
|
+ this.setName("IPC Server handler "+ instanceNumber +
|
|
|
+ " on default port " + port);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -2773,6 +2818,7 @@ public abstract class Server {
|
|
|
this.handlerCount = handlerCount;
|
|
|
this.socketSendBufferSize = 0;
|
|
|
this.serverName = serverName;
|
|
|
+ this.auxiliaryListenerMap = null;
|
|
|
this.maxDataLength = conf.getInt(CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH,
|
|
|
CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
|
|
|
if (queueSizePerHandler != -1) {
|
|
@@ -2812,8 +2858,9 @@ public abstract class Server {
|
|
|
this.negotiateResponse = buildNegotiateResponse(enabledAuthMethods);
|
|
|
|
|
|
// Start the listener here and let it bind to the port
|
|
|
- listener = new Listener();
|
|
|
- this.port = listener.getAddress().getPort();
|
|
|
+ listener = new Listener(port);
|
|
|
+ // set the server port to the default listener port.
|
|
|
+ this.port = listener.getAddress().getPort();
|
|
|
connectionManager = new ConnectionManager();
|
|
|
this.rpcMetrics = RpcMetrics.create(this, conf);
|
|
|
this.rpcDetailedMetrics = RpcDetailedMetrics.create(this.port);
|
|
@@ -2835,7 +2882,23 @@ public abstract class Server {
|
|
|
|
|
|
this.exceptionsHandler.addTerseLoggingExceptions(StandbyException.class);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ public synchronized void addAuxiliaryListener(int auxiliaryPort)
|
|
|
+ throws IOException {
|
|
|
+ if (auxiliaryListenerMap == null) {
|
|
|
+ auxiliaryListenerMap = new HashMap<>();
|
|
|
+ }
|
|
|
+ if (auxiliaryListenerMap.containsKey(auxiliaryPort) && auxiliaryPort != 0) {
|
|
|
+ throw new IOException(
|
|
|
+ "There is already a listener binding to: " + auxiliaryPort);
|
|
|
+ }
|
|
|
+ Listener newListener = new Listener(auxiliaryPort);
|
|
|
+ // in the case of port = 0, the listener would be on a != 0 port.
|
|
|
+ LOG.info("Adding a server listener on port " +
|
|
|
+ newListener.getAddress().getPort());
|
|
|
+ auxiliaryListenerMap.put(newListener.getAddress().getPort(), newListener);
|
|
|
+ }
|
|
|
+
|
|
|
private RpcSaslProto buildNegotiateResponse(List<AuthMethod> authMethods)
|
|
|
throws IOException {
|
|
|
RpcSaslProto.Builder negotiateBuilder = RpcSaslProto.newBuilder();
|
|
@@ -3069,6 +3132,12 @@ public abstract class Server {
|
|
|
public synchronized void start() {
|
|
|
responder.start();
|
|
|
listener.start();
|
|
|
+ if (auxiliaryListenerMap != null && auxiliaryListenerMap.size() > 0) {
|
|
|
+ for (Listener newListener : auxiliaryListenerMap.values()) {
|
|
|
+ newListener.start();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
handlers = new Handler[handlerCount];
|
|
|
|
|
|
for (int i = 0; i < handlerCount; i++) {
|
|
@@ -3090,6 +3159,12 @@ public abstract class Server {
|
|
|
}
|
|
|
listener.interrupt();
|
|
|
listener.doStop();
|
|
|
+ if (auxiliaryListenerMap != null && auxiliaryListenerMap.size() > 0) {
|
|
|
+ for (Listener newListener : auxiliaryListenerMap.values()) {
|
|
|
+ newListener.interrupt();
|
|
|
+ newListener.doStop();
|
|
|
+ }
|
|
|
+ }
|
|
|
responder.interrupt();
|
|
|
notifyAll();
|
|
|
this.rpcMetrics.shutdown();
|
|
@@ -3113,6 +3188,23 @@ public abstract class Server {
|
|
|
public synchronized InetSocketAddress getListenerAddress() {
|
|
|
return listener.getAddress();
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Return the set of all the configured auxiliary socket addresses NameNode
|
|
|
+ * RPC is listening on. If there are none, or it is not configured at all, an
|
|
|
+ * empty set is returned.
|
|
|
+ * @return the set of all the auxiliary addresses on which the
|
|
|
+ * RPC server is listening on.
|
|
|
+ */
|
|
|
+ public synchronized Set<InetSocketAddress> getAuxiliaryListenerAddresses() {
|
|
|
+ Set<InetSocketAddress> allAddrs = new HashSet<>();
|
|
|
+ if (auxiliaryListenerMap != null && auxiliaryListenerMap.size() > 0) {
|
|
|
+ for (Listener auxListener : auxiliaryListenerMap.values()) {
|
|
|
+ allAddrs.add(auxListener.getAddress());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return allAddrs;
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Called for each call.
|
|
@@ -3417,11 +3509,11 @@ public abstract class Server {
|
|
|
return connections.toArray(new Connection[0]);
|
|
|
}
|
|
|
|
|
|
- Connection register(SocketChannel channel) {
|
|
|
+ Connection register(SocketChannel channel, int ingressPort) {
|
|
|
if (isFull()) {
|
|
|
return null;
|
|
|
}
|
|
|
- Connection connection = new Connection(channel, Time.now());
|
|
|
+ Connection connection = new Connection(channel, Time.now(), ingressPort);
|
|
|
add(connection);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Server connection from " + connection +
|