|
@@ -20,9 +20,6 @@ package org.apache.hadoop.ipc;
|
|
|
|
|
|
import java.lang.reflect.Proxy;
|
|
|
import java.lang.reflect.Method;
|
|
|
-import java.lang.reflect.Array;
|
|
|
-import java.lang.reflect.InvocationHandler;
|
|
|
-import java.lang.reflect.InvocationTargetException;
|
|
|
|
|
|
import java.net.ConnectException;
|
|
|
import java.net.InetSocketAddress;
|
|
@@ -32,7 +29,6 @@ import java.util.Map;
|
|
|
import java.util.HashMap;
|
|
|
|
|
|
import javax.net.SocketFactory;
|
|
|
-import javax.security.auth.Subject;
|
|
|
import javax.security.auth.login.LoginException;
|
|
|
|
|
|
import org.apache.commons.logging.*;
|
|
@@ -44,6 +40,7 @@ import org.apache.hadoop.security.authorize.AuthorizationException;
|
|
|
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
|
|
|
import org.apache.hadoop.conf.*;
|
|
|
import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
|
|
|
+import org.apache.hadoop.util.ReflectionUtils;
|
|
|
|
|
|
/** A simple RPC mechanism.
|
|
|
*
|
|
@@ -64,185 +61,55 @@ import org.apache.hadoop.metrics.util.MetricsTimeVaryingRate;
|
|
|
* the protocol instance is transmitted.
|
|
|
*/
|
|
|
public class RPC {
|
|
|
- private static final Log LOG =
|
|
|
- LogFactory.getLog(RPC.class);
|
|
|
+ private static final Log LOG = LogFactory.getLog(RPC.class);
|
|
|
|
|
|
private RPC() {} // no public ctor
|
|
|
|
|
|
+ // cache of RpcEngines by protocol
|
|
|
+ private static final Map<Class,RpcEngine> PROTOCOL_ENGINES
|
|
|
+ = new HashMap<Class,RpcEngine>();
|
|
|
|
|
|
- /** A method invocation, including the method name and its parameters.*/
|
|
|
- private static class Invocation implements Writable, Configurable {
|
|
|
- private String methodName;
|
|
|
- private Class[] parameterClasses;
|
|
|
- private Object[] parameters;
|
|
|
- private Configuration conf;
|
|
|
+ // track what RpcEngine is used by a proxy class, for stopProxy()
|
|
|
+ private static final Map<Class,RpcEngine> PROXY_ENGINES
|
|
|
+ = new HashMap<Class,RpcEngine>();
|
|
|
|
|
|
- public Invocation() {}
|
|
|
-
|
|
|
- public Invocation(Method method, Object[] parameters) {
|
|
|
- this.methodName = method.getName();
|
|
|
- this.parameterClasses = method.getParameterTypes();
|
|
|
- this.parameters = parameters;
|
|
|
- }
|
|
|
-
|
|
|
- /** The name of the method invoked. */
|
|
|
- public String getMethodName() { return methodName; }
|
|
|
-
|
|
|
- /** The parameter classes. */
|
|
|
- public Class[] getParameterClasses() { return parameterClasses; }
|
|
|
-
|
|
|
- /** The parameter instances. */
|
|
|
- public Object[] getParameters() { return parameters; }
|
|
|
-
|
|
|
- public void readFields(DataInput in) throws IOException {
|
|
|
- methodName = UTF8.readString(in);
|
|
|
- parameters = new Object[in.readInt()];
|
|
|
- parameterClasses = new Class[parameters.length];
|
|
|
- ObjectWritable objectWritable = new ObjectWritable();
|
|
|
- for (int i = 0; i < parameters.length; i++) {
|
|
|
- parameters[i] = ObjectWritable.readObject(in, objectWritable, this.conf);
|
|
|
- parameterClasses[i] = objectWritable.getDeclaredClass();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public void write(DataOutput out) throws IOException {
|
|
|
- UTF8.writeString(out, methodName);
|
|
|
- out.writeInt(parameterClasses.length);
|
|
|
- for (int i = 0; i < parameterClasses.length; i++) {
|
|
|
- ObjectWritable.writeObject(out, parameters[i], parameterClasses[i],
|
|
|
- conf);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- public String toString() {
|
|
|
- StringBuffer buffer = new StringBuffer();
|
|
|
- buffer.append(methodName);
|
|
|
- buffer.append("(");
|
|
|
- for (int i = 0; i < parameters.length; i++) {
|
|
|
- if (i != 0)
|
|
|
- buffer.append(", ");
|
|
|
- buffer.append(parameters[i]);
|
|
|
- }
|
|
|
- buffer.append(")");
|
|
|
- return buffer.toString();
|
|
|
- }
|
|
|
-
|
|
|
- public void setConf(Configuration conf) {
|
|
|
- this.conf = conf;
|
|
|
- }
|
|
|
-
|
|
|
- public Configuration getConf() {
|
|
|
- return this.conf;
|
|
|
- }
|
|
|
+ private static final String ENGINE_PROP = "rpc.engine";
|
|
|
|
|
|
+ // set a protocol to use a non-default RpcEngine
|
|
|
+ static void setProtocolEngine(Configuration conf,
|
|
|
+ Class protocol, Class engine) {
|
|
|
+ conf.setClass(ENGINE_PROP+"."+protocol.getName(), engine, RpcEngine.class);
|
|
|
}
|
|
|
|
|
|
- /* Cache a client using its socket factory as the hash key */
|
|
|
- static private class ClientCache {
|
|
|
- private Map<SocketFactory, Client> clients =
|
|
|
- new HashMap<SocketFactory, Client>();
|
|
|
-
|
|
|
- /**
|
|
|
- * Construct & cache an IPC client with the user-provided SocketFactory
|
|
|
- * if no cached client exists.
|
|
|
- *
|
|
|
- * @param conf Configuration
|
|
|
- * @return an IPC client
|
|
|
- */
|
|
|
- private synchronized Client getClient(Configuration conf,
|
|
|
- SocketFactory factory) {
|
|
|
- // Construct & cache client. The configuration is only used for timeout,
|
|
|
- // and Clients have connection pools. So we can either (a) lose some
|
|
|
- // connection pooling and leak sockets, or (b) use the same timeout for all
|
|
|
- // configurations. Since the IPC is usually intended globally, not
|
|
|
- // per-job, we choose (a).
|
|
|
- Client client = clients.get(factory);
|
|
|
- if (client == null) {
|
|
|
- client = new Client(ObjectWritable.class, conf, factory);
|
|
|
- clients.put(factory, client);
|
|
|
- } else {
|
|
|
- client.incCount();
|
|
|
- }
|
|
|
- return client;
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Construct & cache an IPC client with the default SocketFactory
|
|
|
- * if no cached client exists.
|
|
|
- *
|
|
|
- * @param conf Configuration
|
|
|
- * @return an IPC client
|
|
|
- */
|
|
|
- private synchronized Client getClient(Configuration conf) {
|
|
|
- return getClient(conf, SocketFactory.getDefault());
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Stop a RPC client connection
|
|
|
- * A RPC client is closed only when its reference count becomes zero.
|
|
|
- */
|
|
|
- private void stopClient(Client client) {
|
|
|
- synchronized (this) {
|
|
|
- client.decCount();
|
|
|
- if (client.isZeroReference()) {
|
|
|
- clients.remove(client.getSocketFactory());
|
|
|
- }
|
|
|
- }
|
|
|
- if (client.isZeroReference()) {
|
|
|
- client.stop();
|
|
|
- }
|
|
|
+ // return the RpcEngine configured to handle a protocol
|
|
|
+ private static synchronized RpcEngine getProtocolEngine(Class protocol,
|
|
|
+ Configuration conf) {
|
|
|
+ RpcEngine engine = PROTOCOL_ENGINES.get(protocol);
|
|
|
+ if (engine == null) {
|
|
|
+ Class<?> impl = conf.getClass(ENGINE_PROP+"."+protocol.getName(),
|
|
|
+ WritableRpcEngine.class);
|
|
|
+ LOG.info("Using "+impl.getName()+" for "+protocol.getName());
|
|
|
+ engine = (RpcEngine)ReflectionUtils.newInstance(impl, conf);
|
|
|
+ if (protocol.isInterface())
|
|
|
+ PROXY_ENGINES.put(Proxy.getProxyClass(protocol.getClassLoader(),
|
|
|
+ protocol),
|
|
|
+ engine);
|
|
|
+ PROTOCOL_ENGINES.put(protocol, engine);
|
|
|
}
|
|
|
+ return engine;
|
|
|
}
|
|
|
|
|
|
- private static ClientCache CLIENTS=new ClientCache();
|
|
|
-
|
|
|
- private static class Invoker implements InvocationHandler {
|
|
|
- private Class<? extends VersionedProtocol> protocol;
|
|
|
- private InetSocketAddress address;
|
|
|
- private UserGroupInformation ticket;
|
|
|
- private Client client;
|
|
|
- private boolean isClosed = false;
|
|
|
-
|
|
|
- public Invoker(Class<? extends VersionedProtocol> protocol,
|
|
|
- InetSocketAddress address, UserGroupInformation ticket,
|
|
|
- Configuration conf, SocketFactory factory) {
|
|
|
- this.protocol = protocol;
|
|
|
- this.address = address;
|
|
|
- this.ticket = ticket;
|
|
|
- this.client = CLIENTS.getClient(conf, factory);
|
|
|
- }
|
|
|
-
|
|
|
- public Object invoke(Object proxy, Method method, Object[] args)
|
|
|
- throws Throwable {
|
|
|
- final boolean logDebug = LOG.isDebugEnabled();
|
|
|
- long startTime = 0;
|
|
|
- if (logDebug) {
|
|
|
- startTime = System.currentTimeMillis();
|
|
|
- }
|
|
|
-
|
|
|
- ObjectWritable value = (ObjectWritable)
|
|
|
- client.call(new Invocation(method, args), address,
|
|
|
- protocol, ticket);
|
|
|
- if (logDebug) {
|
|
|
- long callTime = System.currentTimeMillis() - startTime;
|
|
|
- LOG.debug("Call: " + method.getName() + " " + callTime);
|
|
|
- }
|
|
|
- return value.get();
|
|
|
- }
|
|
|
-
|
|
|
- /* close the IPC client that's responsible for this invoker's RPCs */
|
|
|
- synchronized private void close() {
|
|
|
- if (!isClosed) {
|
|
|
- isClosed = true;
|
|
|
- CLIENTS.stopClient(client);
|
|
|
- }
|
|
|
- }
|
|
|
+ // return the RpcEngine that handles a proxy object
|
|
|
+ private static synchronized RpcEngine getProxyEngine(Object proxy) {
|
|
|
+ return PROXY_ENGINES.get(proxy.getClass());
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* A version mismatch for the RPC protocol.
|
|
|
*/
|
|
|
public static class VersionMismatch extends IOException {
|
|
|
+ private static final long serialVersionUID = 0;
|
|
|
+
|
|
|
private String interfaceName;
|
|
|
private long clientVersion;
|
|
|
private long serverVersion;
|
|
@@ -286,8 +153,8 @@ public class RPC {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public static VersionedProtocol waitForProxy(
|
|
|
- Class<? extends VersionedProtocol> protocol,
|
|
|
+ public static Object waitForProxy(
|
|
|
+ Class protocol,
|
|
|
long clientVersion,
|
|
|
InetSocketAddress addr,
|
|
|
Configuration conf
|
|
@@ -305,13 +172,9 @@ public class RPC {
|
|
|
* @return the proxy
|
|
|
* @throws IOException if the far end through a RemoteException
|
|
|
*/
|
|
|
- static VersionedProtocol waitForProxy(
|
|
|
- Class<? extends VersionedProtocol> protocol,
|
|
|
- long clientVersion,
|
|
|
- InetSocketAddress addr,
|
|
|
- Configuration conf,
|
|
|
- long timeout
|
|
|
- ) throws IOException {
|
|
|
+ static Object waitForProxy(Class protocol, long clientVersion,
|
|
|
+ InetSocketAddress addr, Configuration conf,
|
|
|
+ long timeout) throws IOException {
|
|
|
long startTime = System.currentTimeMillis();
|
|
|
IOException ioe;
|
|
|
while (true) {
|
|
@@ -337,12 +200,12 @@ public class RPC {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
/** Construct a client-side proxy object that implements the named protocol,
|
|
|
* talking to a server at the named address. */
|
|
|
- public static VersionedProtocol getProxy(
|
|
|
- Class<? extends VersionedProtocol> protocol,
|
|
|
- long clientVersion, InetSocketAddress addr, Configuration conf,
|
|
|
- SocketFactory factory) throws IOException {
|
|
|
+ public static Object getProxy(Class protocol, long clientVersion,
|
|
|
+ InetSocketAddress addr, Configuration conf,
|
|
|
+ SocketFactory factory) throws IOException {
|
|
|
UserGroupInformation ugi = null;
|
|
|
try {
|
|
|
ugi = UserGroupInformation.login(conf);
|
|
@@ -354,23 +217,13 @@ public class RPC {
|
|
|
|
|
|
/** Construct a client-side proxy object that implements the named protocol,
|
|
|
* talking to a server at the named address. */
|
|
|
- public static VersionedProtocol getProxy(
|
|
|
- Class<? extends VersionedProtocol> protocol,
|
|
|
- long clientVersion, InetSocketAddress addr, UserGroupInformation ticket,
|
|
|
- Configuration conf, SocketFactory factory) throws IOException {
|
|
|
-
|
|
|
- VersionedProtocol proxy =
|
|
|
- (VersionedProtocol) Proxy.newProxyInstance(
|
|
|
- protocol.getClassLoader(), new Class[] { protocol },
|
|
|
- new Invoker(protocol, addr, ticket, conf, factory));
|
|
|
- long serverVersion = proxy.getProtocolVersion(protocol.getName(),
|
|
|
- clientVersion);
|
|
|
- if (serverVersion == clientVersion) {
|
|
|
- return proxy;
|
|
|
- } else {
|
|
|
- throw new VersionMismatch(protocol.getName(), clientVersion,
|
|
|
- serverVersion);
|
|
|
- }
|
|
|
+ public static Object getProxy(Class protocol, long clientVersion,
|
|
|
+ InetSocketAddress addr,
|
|
|
+ UserGroupInformation ticket,
|
|
|
+ Configuration conf,
|
|
|
+ SocketFactory factory) throws IOException {
|
|
|
+ return getProtocolEngine(protocol,conf)
|
|
|
+ .getProxy(protocol, clientVersion, addr, ticket, conf, factory);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -383,10 +236,9 @@ public class RPC {
|
|
|
* @return a proxy instance
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public static VersionedProtocol getProxy(
|
|
|
- Class<? extends VersionedProtocol> protocol,
|
|
|
- long clientVersion, InetSocketAddress addr, Configuration conf)
|
|
|
- throws IOException {
|
|
|
+ public static Object getProxy(Class protocol, long clientVersion,
|
|
|
+ InetSocketAddress addr, Configuration conf)
|
|
|
+ throws IOException {
|
|
|
|
|
|
return getProxy(protocol, clientVersion, addr, conf, NetUtils
|
|
|
.getDefaultSocketFactory(conf));
|
|
@@ -396,9 +248,9 @@ public class RPC {
|
|
|
* Stop this proxy and release its invoker's resource
|
|
|
* @param proxy the proxy to be stopped
|
|
|
*/
|
|
|
- public static void stopProxy(VersionedProtocol proxy) {
|
|
|
+ public static void stopProxy(Object proxy) {
|
|
|
if (proxy!=null) {
|
|
|
- ((Invoker)Proxy.getInvocationHandler(proxy)).close();
|
|
|
+ getProxyEngine(proxy).stopProxy(proxy);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -406,6 +258,7 @@ public class RPC {
|
|
|
* Expert: Make multiple, parallel calls to a set of servers.
|
|
|
* @deprecated Use {@link #call(Method, Object[][], InetSocketAddress[], UserGroupInformation, Configuration)} instead
|
|
|
*/
|
|
|
+ @Deprecated
|
|
|
public static Object[] call(Method method, Object[][] params,
|
|
|
InetSocketAddress[] addrs, Configuration conf)
|
|
|
throws IOException {
|
|
@@ -418,169 +271,61 @@ public class RPC {
|
|
|
UserGroupInformation ticket, Configuration conf)
|
|
|
throws IOException {
|
|
|
|
|
|
- Invocation[] invocations = new Invocation[params.length];
|
|
|
- for (int i = 0; i < params.length; i++)
|
|
|
- invocations[i] = new Invocation(method, params[i]);
|
|
|
- Client client = CLIENTS.getClient(conf);
|
|
|
- try {
|
|
|
- Writable[] wrappedValues =
|
|
|
- client.call(invocations, addrs, method.getDeclaringClass(), ticket);
|
|
|
-
|
|
|
- if (method.getReturnType() == Void.TYPE) {
|
|
|
- return null;
|
|
|
- }
|
|
|
-
|
|
|
- Object[] values =
|
|
|
- (Object[])Array.newInstance(method.getReturnType(), wrappedValues.length);
|
|
|
- for (int i = 0; i < values.length; i++)
|
|
|
- if (wrappedValues[i] != null)
|
|
|
- values[i] = ((ObjectWritable)wrappedValues[i]).get();
|
|
|
-
|
|
|
- return values;
|
|
|
- } finally {
|
|
|
- CLIENTS.stopClient(client);
|
|
|
- }
|
|
|
+ return getProtocolEngine(method.getDeclaringClass(), conf)
|
|
|
+ .call(method, params, addrs, ticket, conf);
|
|
|
}
|
|
|
|
|
|
/** Construct a server for a protocol implementation instance listening on a
|
|
|
- * port and address. */
|
|
|
+ * port and address.
|
|
|
+ * @deprecated protocol interface should be passed.
|
|
|
+ */
|
|
|
+ @Deprecated
|
|
|
public static Server getServer(final Object instance, final String bindAddress, final int port, Configuration conf)
|
|
|
throws IOException {
|
|
|
return getServer(instance, bindAddress, port, 1, false, conf);
|
|
|
}
|
|
|
|
|
|
/** Construct a server for a protocol implementation instance listening on a
|
|
|
- * port and address. */
|
|
|
+ * port and address.
|
|
|
+ * @deprecated protocol interface should be passed.
|
|
|
+ */
|
|
|
+ @Deprecated
|
|
|
public static Server getServer(final Object instance, final String bindAddress, final int port,
|
|
|
final int numHandlers,
|
|
|
final boolean verbose, Configuration conf)
|
|
|
throws IOException {
|
|
|
- return new Server(instance, conf, bindAddress, port, numHandlers, verbose);
|
|
|
+ return getServer(instance.getClass(), // use impl class for protocol
|
|
|
+ instance, bindAddress, port, numHandlers, false, conf);
|
|
|
}
|
|
|
|
|
|
- /** An RPC Server. */
|
|
|
- public static class Server extends org.apache.hadoop.ipc.Server {
|
|
|
- private Object instance;
|
|
|
- private boolean verbose;
|
|
|
- private boolean authorize = false;
|
|
|
-
|
|
|
- /** Construct an RPC server.
|
|
|
- * @param instance the instance whose methods will be called
|
|
|
- * @param conf the configuration to use
|
|
|
- * @param bindAddress the address to bind on to listen for connection
|
|
|
- * @param port the port to listen for connections on
|
|
|
- */
|
|
|
- public Server(Object instance, Configuration conf, String bindAddress, int port)
|
|
|
- throws IOException {
|
|
|
- this(instance, conf, bindAddress, port, 1, false);
|
|
|
- }
|
|
|
-
|
|
|
- private static String classNameBase(String className) {
|
|
|
- String[] names = className.split("\\.", -1);
|
|
|
- if (names == null || names.length == 0) {
|
|
|
- return className;
|
|
|
- }
|
|
|
- return names[names.length-1];
|
|
|
- }
|
|
|
-
|
|
|
- /** Construct an RPC server.
|
|
|
- * @param instance the instance whose methods will be called
|
|
|
- * @param conf the configuration to use
|
|
|
- * @param bindAddress the address to bind on to listen for connection
|
|
|
- * @param port the port to listen for connections on
|
|
|
- * @param numHandlers the number of method handler threads to run
|
|
|
- * @param verbose whether each call should be logged
|
|
|
- */
|
|
|
- public Server(Object instance, Configuration conf, String bindAddress, int port,
|
|
|
- int numHandlers, boolean verbose) throws IOException {
|
|
|
- super(bindAddress, port, Invocation.class, numHandlers, conf, classNameBase(instance.getClass().getName()));
|
|
|
- this.instance = instance;
|
|
|
- this.verbose = verbose;
|
|
|
- this.authorize =
|
|
|
- conf.getBoolean(ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG,
|
|
|
- false);
|
|
|
- }
|
|
|
-
|
|
|
- public Writable call(Class<?> protocol, Writable param, long receivedTime)
|
|
|
+ /** Construct a server for a protocol implementation instance. */
|
|
|
+ public static Server getServer(Class protocol,
|
|
|
+ Object instance, String bindAddress,
|
|
|
+ int port, Configuration conf)
|
|
|
throws IOException {
|
|
|
- try {
|
|
|
- Invocation call = (Invocation)param;
|
|
|
- if (verbose) log("Call: " + call);
|
|
|
-
|
|
|
- Method method =
|
|
|
- protocol.getMethod(call.getMethodName(),
|
|
|
- call.getParameterClasses());
|
|
|
- method.setAccessible(true);
|
|
|
-
|
|
|
- long startTime = System.currentTimeMillis();
|
|
|
- Object value = method.invoke(instance, call.getParameters());
|
|
|
- int processingTime = (int) (System.currentTimeMillis() - startTime);
|
|
|
- int qTime = (int) (startTime-receivedTime);
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Served: " + call.getMethodName() +
|
|
|
- " queueTime= " + qTime +
|
|
|
- " procesingTime= " + processingTime);
|
|
|
- }
|
|
|
- rpcMetrics.rpcQueueTime.inc(qTime);
|
|
|
- rpcMetrics.rpcProcessingTime.inc(processingTime);
|
|
|
-
|
|
|
- MetricsTimeVaryingRate m =
|
|
|
- (MetricsTimeVaryingRate) rpcMetrics.registry.get(call.getMethodName());
|
|
|
- if (m == null) {
|
|
|
- try {
|
|
|
- m = new MetricsTimeVaryingRate(call.getMethodName(),
|
|
|
- rpcMetrics.registry);
|
|
|
- } catch (IllegalArgumentException iae) {
|
|
|
- // the metrics has been registered; re-fetch the handle
|
|
|
- LOG.info("Error register " + call.getMethodName(), iae);
|
|
|
- m = (MetricsTimeVaryingRate) rpcMetrics.registry.get(
|
|
|
- call.getMethodName());
|
|
|
- }
|
|
|
- }
|
|
|
- m.inc(processingTime);
|
|
|
-
|
|
|
- if (verbose) log("Return: "+value);
|
|
|
-
|
|
|
- return new ObjectWritable(method.getReturnType(), value);
|
|
|
+ return getServer(protocol, instance, bindAddress, port, 1, false, conf);
|
|
|
+ }
|
|
|
|
|
|
- } catch (InvocationTargetException e) {
|
|
|
- Throwable target = e.getTargetException();
|
|
|
- if (target instanceof IOException) {
|
|
|
- throw (IOException)target;
|
|
|
- } else {
|
|
|
- IOException ioe = new IOException(target.toString());
|
|
|
- ioe.setStackTrace(target.getStackTrace());
|
|
|
- throw ioe;
|
|
|
- }
|
|
|
- } catch (Throwable e) {
|
|
|
- if (!(e instanceof IOException)) {
|
|
|
- LOG.error("Unexpected throwable object ", e);
|
|
|
- }
|
|
|
- IOException ioe = new IOException(e.toString());
|
|
|
- ioe.setStackTrace(e.getStackTrace());
|
|
|
- throw ioe;
|
|
|
- }
|
|
|
- }
|
|
|
+ /** Construct a server for a protocol implementation instance. */
|
|
|
+ public static Server getServer(Class protocol,
|
|
|
+ Object instance, String bindAddress, int port,
|
|
|
+ int numHandlers,
|
|
|
+ boolean verbose, Configuration conf)
|
|
|
+ throws IOException {
|
|
|
+
|
|
|
+ return getProtocolEngine(protocol, conf)
|
|
|
+ .getServer(protocol, instance, bindAddress, port, numHandlers, verbose,
|
|
|
+ conf);
|
|
|
+ }
|
|
|
|
|
|
- @Override
|
|
|
- public void authorize(Subject user, ConnectionHeader connection)
|
|
|
- throws AuthorizationException {
|
|
|
- if (authorize) {
|
|
|
- Class<?> protocol = null;
|
|
|
- try {
|
|
|
- protocol = getProtocolClass(connection.getProtocol(), getConf());
|
|
|
- } catch (ClassNotFoundException cfne) {
|
|
|
- throw new AuthorizationException("Unknown protocol: " +
|
|
|
- connection.getProtocol());
|
|
|
- }
|
|
|
- ServiceAuthorizationManager.authorize(user, protocol);
|
|
|
- }
|
|
|
+ /** An RPC Server. */
|
|
|
+ public abstract static class Server extends org.apache.hadoop.ipc.Server {
|
|
|
+
|
|
|
+ protected Server(String bindAddress, int port,
|
|
|
+ Class<? extends Writable> paramClass, int handlerCount,
|
|
|
+ Configuration conf, String serverName) throws IOException {
|
|
|
+ super(bindAddress, port, paramClass, handlerCount, conf, serverName);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static void log(String value) {
|
|
|
- if (value!= null && value.length() > 55)
|
|
|
- value = value.substring(0, 55)+"...";
|
|
|
- LOG.info(value);
|
|
|
- }
|
|
|
}
|