|
@@ -257,6 +257,7 @@ public class Client {
|
|
|
private final ConnectionId remoteId; // connection id
|
|
|
private AuthMethod authMethod; // authentication method
|
|
|
private Token<? extends TokenIdentifier> token;
|
|
|
+ private int serviceClass;
|
|
|
private SaslRpcClient saslRpcClient;
|
|
|
|
|
|
private Socket socket = null; // connected socket
|
|
@@ -279,7 +280,7 @@ public class Client {
|
|
|
|
|
|
private final Object sendRpcRequestLock = new Object();
|
|
|
|
|
|
- public Connection(ConnectionId remoteId) throws IOException {
|
|
|
+ public Connection(ConnectionId remoteId, int serviceClass) throws IOException {
|
|
|
this.remoteId = remoteId;
|
|
|
this.server = remoteId.getAddress();
|
|
|
if (server.isUnresolved()) {
|
|
@@ -296,6 +297,7 @@ public class Client {
|
|
|
this.tcpNoDelay = remoteId.getTcpNoDelay();
|
|
|
this.doPing = remoteId.getDoPing();
|
|
|
this.pingInterval = remoteId.getPingInterval();
|
|
|
+ this.serviceClass = serviceClass;
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("The ping interval is " + this.pingInterval + " ms.");
|
|
|
}
|
|
@@ -747,7 +749,9 @@ public class Client {
|
|
|
* +----------------------------------+
|
|
|
* | "hrpc" 4 bytes |
|
|
|
* +----------------------------------+
|
|
|
- * | Version (1 bytes) |
|
|
|
+ * | Version (1 byte) |
|
|
|
+ * +----------------------------------+
|
|
|
+ * | Service Class (1 byte) |
|
|
|
* +----------------------------------+
|
|
|
* | Authmethod (1 byte) |
|
|
|
* +----------------------------------+
|
|
@@ -760,6 +764,7 @@ public class Client {
|
|
|
// Write out the header, version and authentication method
|
|
|
out.write(Server.HEADER.array());
|
|
|
out.write(Server.CURRENT_VERSION);
|
|
|
+ out.write(serviceClass);
|
|
|
authMethod.write(out);
|
|
|
Server.IpcSerializationType.PROTOBUF.write(out);
|
|
|
out.flush();
|
|
@@ -1179,19 +1184,33 @@ public class Client {
|
|
|
|
|
|
|
|
|
/**
|
|
|
- * Same as {@link #call(RPC.RpcKind, Writable, InetSocketAddress,
|
|
|
+ * Same as {@link #call(RPC.RpcKind, Writable, InetSocketAddress,
|
|
|
* Class, UserGroupInformation, int, Configuration)}
|
|
|
* except that rpcKind is writable.
|
|
|
*/
|
|
|
- public Writable call(Writable param, InetSocketAddress addr,
|
|
|
+ public Writable call(Writable param, InetSocketAddress addr,
|
|
|
Class<?> protocol, UserGroupInformation ticket,
|
|
|
- int rpcTimeout, Configuration conf)
|
|
|
+ int rpcTimeout, Configuration conf)
|
|
|
throws InterruptedException, IOException {
|
|
|
- ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
|
|
|
+ ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
|
|
|
ticket, rpcTimeout, conf);
|
|
|
return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Same as {@link #call(Writable, InetSocketAddress,
|
|
|
+ * Class, UserGroupInformation, int, Configuration)}
|
|
|
+ * except that specifying serviceClass.
|
|
|
+ */
|
|
|
+ public Writable call(Writable param, InetSocketAddress addr,
|
|
|
+ Class<?> protocol, UserGroupInformation ticket,
|
|
|
+ int rpcTimeout, int serviceClass, Configuration conf)
|
|
|
+ throws InterruptedException, IOException {
|
|
|
+ ConnectionId remoteId = ConnectionId.getConnectionId(addr, protocol,
|
|
|
+ ticket, rpcTimeout, conf);
|
|
|
+ return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId, serviceClass);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Make a call, passing <code>param</code>, to the IPC server running at
|
|
|
* <code>address</code> which is servicing the <code>protocol</code> protocol,
|
|
@@ -1218,6 +1237,22 @@ public class Client {
|
|
|
return call(RPC.RpcKind.RPC_BUILTIN, param, remoteId);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
|
|
|
+ * <code>remoteId</code>, returning the rpc respond.
|
|
|
+ *
|
|
|
+ * @param rpcKind
|
|
|
+ * @param rpcRequest - contains serialized method and method parameters
|
|
|
+ * @param remoteId - the target rpc server
|
|
|
+ * @returns the rpc response
|
|
|
+ * Throws exceptions if there are network problems or if the remote code
|
|
|
+ * threw an exception.
|
|
|
+ */
|
|
|
+ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
|
|
|
+ ConnectionId remoteId) throws InterruptedException, IOException {
|
|
|
+ return call(rpcKind, rpcRequest, remoteId, RPC.RPC_SERVICE_CLASS_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
|
|
|
* <code>remoteId</code>, returning the rpc respond.
|
|
@@ -1225,14 +1260,16 @@ public class Client {
|
|
|
* @param rpcKind
|
|
|
* @param rpcRequest - contains serialized method and method parameters
|
|
|
* @param remoteId - the target rpc server
|
|
|
+ * @param serviceClass - service class for RPC
|
|
|
* @returns the rpc response
|
|
|
* Throws exceptions if there are network problems or if the remote code
|
|
|
* threw an exception.
|
|
|
*/
|
|
|
public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
|
|
|
- ConnectionId remoteId) throws InterruptedException, IOException {
|
|
|
+ ConnectionId remoteId, int serviceClass)
|
|
|
+ throws InterruptedException, IOException {
|
|
|
Call call = new Call(rpcKind, rpcRequest);
|
|
|
- Connection connection = getConnection(remoteId, call);
|
|
|
+ Connection connection = getConnection(remoteId, call, serviceClass);
|
|
|
try {
|
|
|
connection.sendRpcRequest(call); // send the rpc request
|
|
|
} catch (RejectedExecutionException e) {
|
|
@@ -1289,7 +1326,7 @@ public class Client {
|
|
|
/** Get a connection from the pool, or create a new one and add it to the
|
|
|
* pool. Connections to a given ConnectionId are reused. */
|
|
|
private Connection getConnection(ConnectionId remoteId,
|
|
|
- Call call)
|
|
|
+ Call call, int serviceClass)
|
|
|
throws IOException, InterruptedException {
|
|
|
if (!running.get()) {
|
|
|
// the client is stopped
|
|
@@ -1304,7 +1341,7 @@ public class Client {
|
|
|
synchronized (connections) {
|
|
|
connection = connections.get(remoteId);
|
|
|
if (connection == null) {
|
|
|
- connection = new Connection(remoteId);
|
|
|
+ connection = new Connection(remoteId, serviceClass);
|
|
|
connections.put(remoteId, connection);
|
|
|
}
|
|
|
}
|