|
@@ -27,29 +27,22 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.io.DataOutputOutputStream;
|
|
|
import org.apache.hadoop.io.Writable;
|
|
|
import org.apache.hadoop.io.retry.RetryPolicy;
|
|
|
import org.apache.hadoop.ipc.Client.ConnectionId;
|
|
|
import org.apache.hadoop.ipc.RPC.RpcInvoker;
|
|
|
import org.apache.hadoop.ipc.RpcWritable;
|
|
|
import org.apache.hadoop.ipc.protobuf.ProtobufRpcEngineProtos.RequestHeaderProto;
|
|
|
-import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
|
|
|
-import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.SecretManager;
|
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
|
-import org.apache.hadoop.util.ProtoUtil;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.util.concurrent.AsyncGet;
|
|
|
import org.apache.htrace.core.TraceScope;
|
|
|
import org.apache.htrace.core.Tracer;
|
|
|
|
|
|
import javax.net.SocketFactory;
|
|
|
-import java.io.DataInput;
|
|
|
-import java.io.DataOutput;
|
|
|
import java.io.IOException;
|
|
|
-import java.io.OutputStream;
|
|
|
import java.lang.reflect.Method;
|
|
|
import java.lang.reflect.Proxy;
|
|
|
import java.net.InetSocketAddress;
|
|
@@ -146,7 +139,7 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
private Invoker(Class<?> protocol, Client.ConnectionId connId,
|
|
|
Configuration conf, SocketFactory factory) {
|
|
|
this.remoteId = connId;
|
|
|
- this.client = CLIENTS.getClient(conf, factory, RpcResponseWrapper.class);
|
|
|
+ this.client = CLIENTS.getClient(conf, factory, RpcWritable.Buffer.class);
|
|
|
this.protocolName = RPC.getProtocolName(protocol);
|
|
|
this.clientProtocolVersion = RPC
|
|
|
.getProtocolVersion(protocol);
|
|
@@ -193,7 +186,7 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
* the server.
|
|
|
*/
|
|
|
@Override
|
|
|
- public Object invoke(Object proxy, final Method method, Object[] args)
|
|
|
+ public Message invoke(Object proxy, final Method method, Object[] args)
|
|
|
throws ServiceException {
|
|
|
long startTime = 0;
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -228,11 +221,11 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
}
|
|
|
|
|
|
|
|
|
- Message theRequest = (Message) args[1];
|
|
|
- final RpcResponseWrapper val;
|
|
|
+ final Message theRequest = (Message) args[1];
|
|
|
+ final RpcWritable.Buffer val;
|
|
|
try {
|
|
|
- val = (RpcResponseWrapper) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
|
|
|
- new RpcRequestWrapper(rpcRequestHeader, theRequest), remoteId,
|
|
|
+ val = (RpcWritable.Buffer) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
|
|
|
+ new RpcProtobufRequest(rpcRequestHeader, theRequest), remoteId,
|
|
|
fallbackToSimpleAuth);
|
|
|
|
|
|
} catch (Throwable e) {
|
|
@@ -256,7 +249,7 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
}
|
|
|
|
|
|
if (Client.isAsynchronousMode()) {
|
|
|
- final AsyncGet<RpcResponseWrapper, IOException> arr
|
|
|
+ final AsyncGet<RpcWritable.Buffer, IOException> arr
|
|
|
= Client.getAsyncRpcResponse();
|
|
|
final AsyncGet<Message, Exception> asyncGet
|
|
|
= new AsyncGet<Message, Exception>() {
|
|
@@ -278,7 +271,7 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
}
|
|
|
|
|
|
private Message getReturnMessage(final Method method,
|
|
|
- final RpcResponseWrapper rrw) throws ServiceException {
|
|
|
+ final RpcWritable.Buffer buf) throws ServiceException {
|
|
|
Message prototype = null;
|
|
|
try {
|
|
|
prototype = getReturnProtoType(method);
|
|
@@ -287,8 +280,7 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
}
|
|
|
Message returnMessage;
|
|
|
try {
|
|
|
- returnMessage = prototype.newBuilderForType()
|
|
|
- .mergeFrom(rrw.theResponseRead).build();
|
|
|
+ returnMessage = buf.getValue(prototype.getDefaultInstanceForType());
|
|
|
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
LOG.trace(Thread.currentThread().getId() + ": Response <- " +
|
|
@@ -329,201 +321,12 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- interface RpcWrapper extends Writable {
|
|
|
- int getLength();
|
|
|
- }
|
|
|
- /**
|
|
|
- * Wrapper for Protocol Buffer Requests
|
|
|
- *
|
|
|
- * Note while this wrapper is writable, the request on the wire is in
|
|
|
- * Protobuf. Several methods on {@link org.apache.hadoop.ipc.Server and RPC}
|
|
|
- * use type Writable as a wrapper to work across multiple RpcEngine kinds.
|
|
|
- */
|
|
|
- private static abstract class RpcMessageWithHeader<T extends GeneratedMessage>
|
|
|
- implements RpcWrapper {
|
|
|
- T requestHeader;
|
|
|
- Message theRequest; // for clientSide, the request is here
|
|
|
- byte[] theRequestRead; // for server side, the request is here
|
|
|
-
|
|
|
- public RpcMessageWithHeader() {
|
|
|
- }
|
|
|
-
|
|
|
- public RpcMessageWithHeader(T requestHeader, Message theRequest) {
|
|
|
- this.requestHeader = requestHeader;
|
|
|
- this.theRequest = theRequest;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void write(DataOutput out) throws IOException {
|
|
|
- OutputStream os = DataOutputOutputStream.constructOutputStream(out);
|
|
|
-
|
|
|
- ((Message)requestHeader).writeDelimitedTo(os);
|
|
|
- theRequest.writeDelimitedTo(os);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void readFields(DataInput in) throws IOException {
|
|
|
- requestHeader = parseHeaderFrom(readVarintBytes(in));
|
|
|
- theRequestRead = readMessageRequest(in);
|
|
|
- }
|
|
|
-
|
|
|
- abstract T parseHeaderFrom(byte[] bytes) throws IOException;
|
|
|
-
|
|
|
- byte[] readMessageRequest(DataInput in) throws IOException {
|
|
|
- return readVarintBytes(in);
|
|
|
- }
|
|
|
-
|
|
|
- private static byte[] readVarintBytes(DataInput in) throws IOException {
|
|
|
- final int length = ProtoUtil.readRawVarint32(in);
|
|
|
- final byte[] bytes = new byte[length];
|
|
|
- in.readFully(bytes);
|
|
|
- return bytes;
|
|
|
- }
|
|
|
-
|
|
|
- public T getMessageHeader() {
|
|
|
- return requestHeader;
|
|
|
- }
|
|
|
-
|
|
|
- public byte[] getMessageBytes() {
|
|
|
- return theRequestRead;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int getLength() {
|
|
|
- int headerLen = requestHeader.getSerializedSize();
|
|
|
- int reqLen;
|
|
|
- if (theRequest != null) {
|
|
|
- reqLen = theRequest.getSerializedSize();
|
|
|
- } else if (theRequestRead != null ) {
|
|
|
- reqLen = theRequestRead.length;
|
|
|
- } else {
|
|
|
- throw new IllegalArgumentException(
|
|
|
- "getLength on uninitialized RpcWrapper");
|
|
|
- }
|
|
|
- return CodedOutputStream.computeRawVarint32Size(headerLen) + headerLen
|
|
|
- + CodedOutputStream.computeRawVarint32Size(reqLen) + reqLen;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private static class RpcRequestWrapper
|
|
|
- extends RpcMessageWithHeader<RequestHeaderProto> {
|
|
|
- @SuppressWarnings("unused")
|
|
|
- public RpcRequestWrapper() {}
|
|
|
-
|
|
|
- public RpcRequestWrapper(
|
|
|
- RequestHeaderProto requestHeader, Message theRequest) {
|
|
|
- super(requestHeader, theRequest);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- RequestHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
|
|
|
- return RequestHeaderProto.parseFrom(bytes);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public String toString() {
|
|
|
- return requestHeader.getDeclaringClassProtocolName() + "." +
|
|
|
- requestHeader.getMethodName();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @InterfaceAudience.LimitedPrivate({"RPC"})
|
|
|
- public static class RpcRequestMessageWrapper
|
|
|
- extends RpcMessageWithHeader<RpcRequestHeaderProto> {
|
|
|
- public RpcRequestMessageWrapper() {}
|
|
|
-
|
|
|
- public RpcRequestMessageWrapper(
|
|
|
- RpcRequestHeaderProto requestHeader, Message theRequest) {
|
|
|
- super(requestHeader, theRequest);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- RpcRequestHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
|
|
|
- return RpcRequestHeaderProto.parseFrom(bytes);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @InterfaceAudience.LimitedPrivate({"RPC"})
|
|
|
- public static class RpcResponseMessageWrapper
|
|
|
- extends RpcMessageWithHeader<RpcResponseHeaderProto> {
|
|
|
- public RpcResponseMessageWrapper() {}
|
|
|
-
|
|
|
- public RpcResponseMessageWrapper(
|
|
|
- RpcResponseHeaderProto responseHeader, Message theRequest) {
|
|
|
- super(responseHeader, theRequest);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- byte[] readMessageRequest(DataInput in) throws IOException {
|
|
|
- // error message contain no message body
|
|
|
- switch (requestHeader.getStatus()) {
|
|
|
- case ERROR:
|
|
|
- case FATAL:
|
|
|
- return null;
|
|
|
- default:
|
|
|
- return super.readMessageRequest(in);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- RpcResponseHeaderProto parseHeaderFrom(byte[] bytes) throws IOException {
|
|
|
- return RpcResponseHeaderProto.parseFrom(bytes);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Wrapper for Protocol Buffer Responses
|
|
|
- *
|
|
|
- * Note while this wrapper is writable, the request on the wire is in
|
|
|
- * Protobuf. Several methods on {@link org.apache.hadoop.ipc.Server and RPC}
|
|
|
- * use type Writable as a wrapper to work across multiple RpcEngine kinds.
|
|
|
- */
|
|
|
- @InterfaceAudience.LimitedPrivate({"RPC"}) // temporarily exposed
|
|
|
- public static class RpcResponseWrapper implements RpcWrapper {
|
|
|
- Message theResponse; // for senderSide, the response is here
|
|
|
- byte[] theResponseRead; // for receiver side, the response is here
|
|
|
-
|
|
|
- public RpcResponseWrapper() {
|
|
|
- }
|
|
|
-
|
|
|
- public RpcResponseWrapper(Message message) {
|
|
|
- this.theResponse = message;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void write(DataOutput out) throws IOException {
|
|
|
- OutputStream os = DataOutputOutputStream.constructOutputStream(out);
|
|
|
- theResponse.writeDelimitedTo(os);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public void readFields(DataInput in) throws IOException {
|
|
|
- int length = ProtoUtil.readRawVarint32(in);
|
|
|
- theResponseRead = new byte[length];
|
|
|
- in.readFully(theResponseRead);
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public int getLength() {
|
|
|
- int resLen;
|
|
|
- if (theResponse != null) {
|
|
|
- resLen = theResponse.getSerializedSize();
|
|
|
- } else if (theResponseRead != null ) {
|
|
|
- resLen = theResponseRead.length;
|
|
|
- } else {
|
|
|
- throw new IllegalArgumentException(
|
|
|
- "getLength on uninitialized RpcWrapper");
|
|
|
- }
|
|
|
- return CodedOutputStream.computeRawVarint32Size(resLen) + resLen;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
@InterfaceAudience.Private
|
|
|
@InterfaceStability.Unstable
|
|
|
static Client getClient(Configuration conf) {
|
|
|
return CLIENTS.getClient(conf, SocketFactory.getDefault(),
|
|
|
- RpcResponseWrapper.class);
|
|
|
+ RpcWritable.Buffer.class);
|
|
|
}
|
|
|
|
|
|
|
|
@@ -691,16 +494,30 @@ public class ProtobufRpcEngine implements RpcEngine {
|
|
|
// which uses the rpc header. in the normal case we want to defer decoding
|
|
|
// the rpc header until needed by the rpc engine.
|
|
|
static class RpcProtobufRequest extends RpcWritable.Buffer {
|
|
|
- private RequestHeaderProto lazyHeader;
|
|
|
+ private volatile RequestHeaderProto requestHeader;
|
|
|
+ private Message payload;
|
|
|
|
|
|
public RpcProtobufRequest() {
|
|
|
}
|
|
|
|
|
|
- synchronized RequestHeaderProto getRequestHeader() throws IOException {
|
|
|
- if (lazyHeader == null) {
|
|
|
- lazyHeader = getValue(RequestHeaderProto.getDefaultInstance());
|
|
|
+ RpcProtobufRequest(RequestHeaderProto header, Message payload) {
|
|
|
+ this.requestHeader = header;
|
|
|
+ this.payload = payload;
|
|
|
+ }
|
|
|
+
|
|
|
+ RequestHeaderProto getRequestHeader() throws IOException {
|
|
|
+ if (getByteBuffer() != null && requestHeader == null) {
|
|
|
+ requestHeader = getValue(RequestHeaderProto.getDefaultInstance());
|
|
|
+ }
|
|
|
+ return requestHeader;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void writeTo(ResponseBuffer out) throws IOException {
|
|
|
+ requestHeader.writeDelimitedTo(out);
|
|
|
+ if (payload != null) {
|
|
|
+ payload.writeDelimitedTo(out);
|
|
|
}
|
|
|
- return lazyHeader;
|
|
|
}
|
|
|
|
|
|
// this is used by htrace to name the span.
|