|
@@ -77,7 +77,6 @@ public class ConnectionPool {
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(ConnectionPool.class);
|
|
|
|
|
|
-
|
|
|
/** Configuration settings for the connection pool. */
|
|
|
private final Configuration conf;
|
|
|
|
|
@@ -94,6 +93,8 @@ public class ConnectionPool {
|
|
|
private volatile List<ConnectionContext> connections = new ArrayList<>();
|
|
|
/** Connection index for round-robin. */
|
|
|
private final AtomicInteger clientIndex = new AtomicInteger(0);
|
|
|
+ /** Underlying socket index. **/
|
|
|
+ private final AtomicInteger socketIndex = new AtomicInteger(0);
|
|
|
|
|
|
/** Min number of connections per user. */
|
|
|
private final int minSize;
|
|
@@ -105,6 +106,9 @@ public class ConnectionPool {
|
|
|
/** The last time a connection was active. */
|
|
|
private volatile long lastActiveTime = 0;
|
|
|
|
|
|
+ /** Enable using multiple physical socket or not. **/
|
|
|
+ private final boolean enableMultiSocket;
|
|
|
+
|
|
|
/** Map for the protocols and their protobuf implementations. */
|
|
|
private final static Map<Class<?>, ProtoImpl> PROTO_MAP = new HashMap<>();
|
|
|
static {
|
|
@@ -149,9 +153,12 @@ public class ConnectionPool {
|
|
|
this.minSize = minPoolSize;
|
|
|
this.maxSize = maxPoolSize;
|
|
|
this.minActiveRatio = minActiveRatio;
|
|
|
+ this.enableMultiSocket = conf.getBoolean(
|
|
|
+ RBFConfigKeys.DFS_ROUTER_NAMENODE_ENABLE_MULTIPLE_SOCKET_KEY,
|
|
|
+ RBFConfigKeys.DFS_ROUTER_NAMENODE_ENABLE_MULTIPLE_SOCKET_DEFAULT);
|
|
|
|
|
|
// Add minimum connections to the pool
|
|
|
- for (int i=0; i<this.minSize; i++) {
|
|
|
+ for (int i = 0; i < this.minSize; i++) {
|
|
|
ConnectionContext newConnection = newConnection();
|
|
|
this.connections.add(newConnection);
|
|
|
}
|
|
@@ -210,24 +217,23 @@ public class ConnectionPool {
|
|
|
* @return Connection context.
|
|
|
*/
|
|
|
protected ConnectionContext getConnection() {
|
|
|
-
|
|
|
this.lastActiveTime = Time.now();
|
|
|
-
|
|
|
- // Get a connection from the pool following round-robin
|
|
|
- ConnectionContext conn = null;
|
|
|
List<ConnectionContext> tmpConnections = this.connections;
|
|
|
- int size = tmpConnections.size();
|
|
|
- // Inc and mask off sign bit, lookup index should be non-negative int
|
|
|
- int threadIndex = this.clientIndex.getAndIncrement() & 0x7FFFFFFF;
|
|
|
- for (int i=0; i<size; i++) {
|
|
|
- int index = (threadIndex + i) % size;
|
|
|
- conn = tmpConnections.get(index);
|
|
|
- if (conn != null && conn.isUsable()) {
|
|
|
- return conn;
|
|
|
+ for (ConnectionContext tmpConnection : tmpConnections) {
|
|
|
+ if (tmpConnection != null && tmpConnection.isUsable()) {
|
|
|
+ return tmpConnection;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // We return a connection even if it's active
|
|
|
+ ConnectionContext conn = null;
|
|
|
+ // We return a connection even if it's busy
|
|
|
+ int size = tmpConnections.size();
|
|
|
+ if (size > 0) {
|
|
|
+ // Get a connection from the pool following round-robin
|
|
|
+ // Inc and mask off sign bit, lookup index should be non-negative int
|
|
|
+ int threadIndex = this.clientIndex.getAndIncrement() & 0x7FFFFFFF;
|
|
|
+ conn = tmpConnections.get(threadIndex % size);
|
|
|
+ }
|
|
|
return conn;
|
|
|
}
|
|
|
|
|
@@ -256,10 +262,9 @@ public class ConnectionPool {
|
|
|
int targetCount = Math.min(num, this.connections.size() - this.minSize);
|
|
|
// Remove and close targetCount of connections
|
|
|
List<ConnectionContext> tmpConnections = new ArrayList<>();
|
|
|
- for (int i = 0; i < this.connections.size(); i++) {
|
|
|
- ConnectionContext conn = this.connections.get(i);
|
|
|
+ for (ConnectionContext conn : this.connections) {
|
|
|
// Only pick idle connections to close
|
|
|
- if (removed.size() < targetCount && conn.isUsable()) {
|
|
|
+ if (removed.size() < targetCount && conn.isIdle()) {
|
|
|
removed.add(conn);
|
|
|
} else {
|
|
|
tmpConnections.add(conn);
|
|
@@ -267,8 +272,8 @@ public class ConnectionPool {
|
|
|
}
|
|
|
this.connections = tmpConnections;
|
|
|
}
|
|
|
- LOG.debug("Expected to remove {} connection " +
|
|
|
- "and actually removed {} connections", num, removed.size());
|
|
|
+ LOG.debug("Expected to remove {} connection and actually removed {} connections",
|
|
|
+ num, removed.size());
|
|
|
return removed;
|
|
|
}
|
|
|
|
|
@@ -303,7 +308,6 @@ public class ConnectionPool {
|
|
|
*/
|
|
|
protected int getNumActiveConnections() {
|
|
|
int ret = 0;
|
|
|
-
|
|
|
List<ConnectionContext> tmpConnections = this.connections;
|
|
|
for (ConnectionContext conn : tmpConnections) {
|
|
|
if (conn.isActive()) {
|
|
@@ -320,10 +324,9 @@ public class ConnectionPool {
|
|
|
*/
|
|
|
protected int getNumIdleConnections() {
|
|
|
int ret = 0;
|
|
|
-
|
|
|
List<ConnectionContext> tmpConnections = this.connections;
|
|
|
for (ConnectionContext conn : tmpConnections) {
|
|
|
- if (conn.isUsable()) {
|
|
|
+ if (conn.isIdle()) {
|
|
|
ret++;
|
|
|
}
|
|
|
}
|
|
@@ -393,8 +396,9 @@ public class ConnectionPool {
|
|
|
* @throws IOException If it cannot get a new connection.
|
|
|
*/
|
|
|
public ConnectionContext newConnection() throws IOException {
|
|
|
- return newConnection(
|
|
|
- this.conf, this.namenodeAddress, this.ugi, this.protocol);
|
|
|
+ return newConnection(this.conf, this.namenodeAddress,
|
|
|
+ this.ugi, this.protocol, this.enableMultiSocket,
|
|
|
+ this.socketIndex.incrementAndGet());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -402,19 +406,20 @@ public class ConnectionPool {
|
|
|
* context for a single user/security context. To maximize throughput it is
|
|
|
* recommended to use multiple connection per user+server, allowing multiple
|
|
|
* writes and reads to be dispatched in parallel.
|
|
|
- * @param <T>
|
|
|
+ * @param <T> Input type T.
|
|
|
*
|
|
|
* @param conf Configuration for the connection.
|
|
|
* @param nnAddress Address of server supporting the ClientProtocol.
|
|
|
* @param ugi User context.
|
|
|
* @param proto Interface of the protocol.
|
|
|
+ * @param enableMultiSocket Enable multiple socket or not.
|
|
|
* @return proto for the target ClientProtocol that contains the user's
|
|
|
* security context.
|
|
|
* @throws IOException If it cannot be created.
|
|
|
*/
|
|
|
protected static <T> ConnectionContext newConnection(Configuration conf,
|
|
|
- String nnAddress, UserGroupInformation ugi, Class<T> proto)
|
|
|
- throws IOException {
|
|
|
+ String nnAddress, UserGroupInformation ugi, Class<T> proto,
|
|
|
+ boolean enableMultiSocket, int socketIndex) throws IOException {
|
|
|
if (!PROTO_MAP.containsKey(proto)) {
|
|
|
String msg = "Unsupported protocol for connection to NameNode: "
|
|
|
+ ((proto != null) ? proto.getName() : "null");
|
|
@@ -437,15 +442,23 @@ public class ConnectionPool {
|
|
|
}
|
|
|
InetSocketAddress socket = NetUtils.createSocketAddr(nnAddress);
|
|
|
final long version = RPC.getProtocolVersion(classes.protoPb);
|
|
|
- Object proxy = RPC.getProtocolProxy(classes.protoPb, version, socket, ugi,
|
|
|
- conf, factory, RPC.getRpcTimeout(conf), defaultPolicy, null).getProxy();
|
|
|
+ Object proxy;
|
|
|
+ if (enableMultiSocket) {
|
|
|
+ FederationConnectionId connectionId = new FederationConnectionId(
|
|
|
+ socket, classes.protoPb, ugi, RPC.getRpcTimeout(conf),
|
|
|
+ defaultPolicy, conf, socketIndex);
|
|
|
+ proxy = RPC.getProtocolProxy(classes.protoPb, version, connectionId,
|
|
|
+ conf, factory).getProxy();
|
|
|
+ } else {
|
|
|
+ proxy = RPC.getProtocolProxy(classes.protoPb, version, socket, ugi,
|
|
|
+ conf, factory, RPC.getRpcTimeout(conf), defaultPolicy, null).getProxy();
|
|
|
+ }
|
|
|
+
|
|
|
T client = newProtoClient(proto, classes, proxy);
|
|
|
Text dtService = SecurityUtil.buildTokenService(socket);
|
|
|
|
|
|
- ProxyAndInfo<T> clientProxy =
|
|
|
- new ProxyAndInfo<T>(client, dtService, socket);
|
|
|
- ConnectionContext connection = new ConnectionContext(clientProxy);
|
|
|
- return connection;
|
|
|
+ ProxyAndInfo<T> clientProxy = new ProxyAndInfo<T>(client, dtService, socket);
|
|
|
+ return new ConnectionContext(clientProxy, conf);
|
|
|
}
|
|
|
|
|
|
private static <T> T newProtoClient(Class<T> proto, ProtoImpl classes,
|
|
@@ -453,7 +466,7 @@ public class ConnectionPool {
|
|
|
try {
|
|
|
Constructor<?> constructor =
|
|
|
classes.protoClientPb.getConstructor(classes.protoPb);
|
|
|
- Object o = constructor.newInstance(new Object[] {proxy});
|
|
|
+ Object o = constructor.newInstance(proxy);
|
|
|
if (proto.isAssignableFrom(o.getClass())) {
|
|
|
@SuppressWarnings("unchecked")
|
|
|
T client = (T) o;
|