|
@@ -25,7 +25,6 @@ import static org.apache.hadoop.ipc.RpcConstants.PING_CALL_ID;
|
|
|
|
|
|
import java.io.ByteArrayInputStream;
|
|
import java.io.ByteArrayInputStream;
|
|
import java.io.ByteArrayOutputStream;
|
|
import java.io.ByteArrayOutputStream;
|
|
-import java.io.DataInputStream;
|
|
|
|
import java.io.DataOutputStream;
|
|
import java.io.DataOutputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.lang.reflect.UndeclaredThrowableException;
|
|
import java.lang.reflect.UndeclaredThrowableException;
|
|
@@ -82,8 +81,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.Writable;
|
|
import org.apache.hadoop.io.Writable;
|
|
import org.apache.hadoop.io.WritableUtils;
|
|
import org.apache.hadoop.io.WritableUtils;
|
|
-import org.apache.hadoop.ipc.ProtobufRpcEngine.RpcResponseWrapper;
|
|
|
|
-import org.apache.hadoop.ipc.ProtobufRpcEngine.RpcWrapper;
|
|
|
|
import org.apache.hadoop.ipc.RPC.RpcInvoker;
|
|
import org.apache.hadoop.ipc.RPC.RpcInvoker;
|
|
import org.apache.hadoop.ipc.RPC.VersionMismatch;
|
|
import org.apache.hadoop.ipc.RPC.VersionMismatch;
|
|
import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
|
|
import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
|
|
@@ -113,7 +110,6 @@ import org.apache.hadoop.security.token.SecretManager;
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
|
import org.apache.hadoop.util.ProtoUtil;
|
|
import org.apache.hadoop.util.ProtoUtil;
|
|
-import org.apache.hadoop.util.ReflectionUtils;
|
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.htrace.core.SpanId;
|
|
import org.apache.htrace.core.SpanId;
|
|
@@ -122,9 +118,7 @@ import org.apache.htrace.core.Tracer;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.protobuf.ByteString;
|
|
import com.google.protobuf.ByteString;
|
|
-import com.google.protobuf.CodedOutputStream;
|
|
|
|
import com.google.protobuf.Message;
|
|
import com.google.protobuf.Message;
|
|
-import com.google.protobuf.Message.Builder;
|
|
|
|
|
|
|
|
/** An abstract IPC service. IPC calls take a single {@link Writable} as a
|
|
/** An abstract IPC service. IPC calls take a single {@link Writable} as a
|
|
* parameter, and return a {@link Writable} as their value. A service runs on
|
|
* parameter, and return a {@link Writable} as their value. A service runs on
|
|
@@ -1345,6 +1339,7 @@ public abstract class Server {
|
|
* A WrappedRpcServerException that is suppressed altogether
|
|
* A WrappedRpcServerException that is suppressed altogether
|
|
* for the purposes of logging.
|
|
* for the purposes of logging.
|
|
*/
|
|
*/
|
|
|
|
+ @SuppressWarnings("serial")
|
|
private static class WrappedRpcServerExceptionSuppressed
|
|
private static class WrappedRpcServerExceptionSuppressed
|
|
extends WrappedRpcServerException {
|
|
extends WrappedRpcServerException {
|
|
public WrappedRpcServerExceptionSuppressed(
|
|
public WrappedRpcServerExceptionSuppressed(
|
|
@@ -1474,10 +1469,10 @@ public abstract class Server {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private void saslReadAndProcess(DataInputStream dis) throws
|
|
|
|
|
|
+ private void saslReadAndProcess(RpcWritable.Buffer buffer) throws
|
|
WrappedRpcServerException, IOException, InterruptedException {
|
|
WrappedRpcServerException, IOException, InterruptedException {
|
|
final RpcSaslProto saslMessage =
|
|
final RpcSaslProto saslMessage =
|
|
- decodeProtobufFromStream(RpcSaslProto.newBuilder(), dis);
|
|
|
|
|
|
+ getMessage(RpcSaslProto.getDefaultInstance(), buffer);
|
|
switch (saslMessage.getState()) {
|
|
switch (saslMessage.getState()) {
|
|
case WRAP: {
|
|
case WRAP: {
|
|
if (!saslContextEstablished || !useWrap) {
|
|
if (!saslContextEstablished || !useWrap) {
|
|
@@ -1675,7 +1670,7 @@ public abstract class Server {
|
|
RpcConstants.INVALID_RETRY_COUNT, null, this);
|
|
RpcConstants.INVALID_RETRY_COUNT, null, this);
|
|
setupResponse(saslCall,
|
|
setupResponse(saslCall,
|
|
RpcStatusProto.SUCCESS, null,
|
|
RpcStatusProto.SUCCESS, null,
|
|
- new RpcResponseWrapper(message), null, null);
|
|
|
|
|
|
+ RpcWritable.wrap(message), null, null);
|
|
saslCall.sendResponse();
|
|
saslCall.sendResponse();
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1780,7 +1775,7 @@ public abstract class Server {
|
|
dataLengthBuffer.clear();
|
|
dataLengthBuffer.clear();
|
|
data.flip();
|
|
data.flip();
|
|
boolean isHeaderRead = connectionContextRead;
|
|
boolean isHeaderRead = connectionContextRead;
|
|
- processOneRpc(data.array());
|
|
|
|
|
|
+ processOneRpc(data);
|
|
data = null;
|
|
data = null;
|
|
if (!isHeaderRead) {
|
|
if (!isHeaderRead) {
|
|
continue;
|
|
continue;
|
|
@@ -1897,7 +1892,7 @@ public abstract class Server {
|
|
* @throws WrappedRpcServerException - if the header cannot be
|
|
* @throws WrappedRpcServerException - if the header cannot be
|
|
* deserialized, or the user is not authorized
|
|
* deserialized, or the user is not authorized
|
|
*/
|
|
*/
|
|
- private void processConnectionContext(DataInputStream dis)
|
|
|
|
|
|
+ private void processConnectionContext(RpcWritable.Buffer buffer)
|
|
throws WrappedRpcServerException {
|
|
throws WrappedRpcServerException {
|
|
// allow only one connection context during a session
|
|
// allow only one connection context during a session
|
|
if (connectionContextRead) {
|
|
if (connectionContextRead) {
|
|
@@ -1905,8 +1900,7 @@ public abstract class Server {
|
|
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
|
|
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
|
|
"Connection context already processed");
|
|
"Connection context already processed");
|
|
}
|
|
}
|
|
- connectionContext = decodeProtobufFromStream(
|
|
|
|
- IpcConnectionContextProto.newBuilder(), dis);
|
|
|
|
|
|
+ connectionContext = getMessage(IpcConnectionContextProto.getDefaultInstance(), buffer);
|
|
protocolName = connectionContext.hasProtocol() ? connectionContext
|
|
protocolName = connectionContext.hasProtocol() ? connectionContext
|
|
.getProtocol() : null;
|
|
.getProtocol() : null;
|
|
|
|
|
|
@@ -1981,7 +1975,7 @@ public abstract class Server {
|
|
if (unwrappedData.remaining() == 0) {
|
|
if (unwrappedData.remaining() == 0) {
|
|
unwrappedDataLengthBuffer.clear();
|
|
unwrappedDataLengthBuffer.clear();
|
|
unwrappedData.flip();
|
|
unwrappedData.flip();
|
|
- processOneRpc(unwrappedData.array());
|
|
|
|
|
|
+ processOneRpc(unwrappedData);
|
|
unwrappedData = null;
|
|
unwrappedData = null;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1997,31 +1991,30 @@ public abstract class Server {
|
|
* the client that does not require verbose logging by the
|
|
* the client that does not require verbose logging by the
|
|
* Listener thread
|
|
* Listener thread
|
|
* @throws InterruptedException
|
|
* @throws InterruptedException
|
|
- */
|
|
|
|
- private void processOneRpc(byte[] buf)
|
|
|
|
|
|
+ */
|
|
|
|
+ private void processOneRpc(ByteBuffer bb)
|
|
throws IOException, WrappedRpcServerException, InterruptedException {
|
|
throws IOException, WrappedRpcServerException, InterruptedException {
|
|
int callId = -1;
|
|
int callId = -1;
|
|
int retry = RpcConstants.INVALID_RETRY_COUNT;
|
|
int retry = RpcConstants.INVALID_RETRY_COUNT;
|
|
try {
|
|
try {
|
|
- final DataInputStream dis =
|
|
|
|
- new DataInputStream(new ByteArrayInputStream(buf));
|
|
|
|
|
|
+ final RpcWritable.Buffer buffer = RpcWritable.Buffer.wrap(bb);
|
|
final RpcRequestHeaderProto header =
|
|
final RpcRequestHeaderProto header =
|
|
- decodeProtobufFromStream(RpcRequestHeaderProto.newBuilder(), dis);
|
|
|
|
|
|
+ getMessage(RpcRequestHeaderProto.getDefaultInstance(), buffer);
|
|
callId = header.getCallId();
|
|
callId = header.getCallId();
|
|
retry = header.getRetryCount();
|
|
retry = header.getRetryCount();
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
LOG.debug(" got #" + callId);
|
|
LOG.debug(" got #" + callId);
|
|
}
|
|
}
|
|
checkRpcHeaders(header);
|
|
checkRpcHeaders(header);
|
|
-
|
|
|
|
|
|
+
|
|
if (callId < 0) { // callIds typically used during connection setup
|
|
if (callId < 0) { // callIds typically used during connection setup
|
|
- processRpcOutOfBandRequest(header, dis);
|
|
|
|
|
|
+ processRpcOutOfBandRequest(header, buffer);
|
|
} else if (!connectionContextRead) {
|
|
} else if (!connectionContextRead) {
|
|
throw new WrappedRpcServerException(
|
|
throw new WrappedRpcServerException(
|
|
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
|
|
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
|
|
"Connection context not established");
|
|
"Connection context not established");
|
|
} else {
|
|
} else {
|
|
- processRpcRequest(header, dis);
|
|
|
|
|
|
+ processRpcRequest(header, buffer);
|
|
}
|
|
}
|
|
} catch (WrappedRpcServerException wrse) { // inform client of error
|
|
} catch (WrappedRpcServerException wrse) { // inform client of error
|
|
Throwable ioe = wrse.getCause();
|
|
Throwable ioe = wrse.getCause();
|
|
@@ -2074,7 +2067,7 @@ public abstract class Server {
|
|
* @throws InterruptedException
|
|
* @throws InterruptedException
|
|
*/
|
|
*/
|
|
private void processRpcRequest(RpcRequestHeaderProto header,
|
|
private void processRpcRequest(RpcRequestHeaderProto header,
|
|
- DataInputStream dis) throws WrappedRpcServerException,
|
|
|
|
|
|
+ RpcWritable.Buffer buffer) throws WrappedRpcServerException,
|
|
InterruptedException {
|
|
InterruptedException {
|
|
Class<? extends Writable> rpcRequestClass =
|
|
Class<? extends Writable> rpcRequestClass =
|
|
getRpcRequestWrapper(header.getRpcKind());
|
|
getRpcRequestWrapper(header.getRpcKind());
|
|
@@ -2088,8 +2081,7 @@ public abstract class Server {
|
|
}
|
|
}
|
|
Writable rpcRequest;
|
|
Writable rpcRequest;
|
|
try { //Read the rpc request
|
|
try { //Read the rpc request
|
|
- rpcRequest = ReflectionUtils.newInstance(rpcRequestClass, conf);
|
|
|
|
- rpcRequest.readFields(dis);
|
|
|
|
|
|
+ rpcRequest = buffer.newInstance(rpcRequestClass, conf);
|
|
} catch (Throwable t) { // includes runtime exception from newInstance
|
|
} catch (Throwable t) { // includes runtime exception from newInstance
|
|
LOG.warn("Unable to read call parameters for client " +
|
|
LOG.warn("Unable to read call parameters for client " +
|
|
getHostAddress() + "on connection protocol " +
|
|
getHostAddress() + "on connection protocol " +
|
|
@@ -2169,8 +2161,8 @@ public abstract class Server {
|
|
* @throws InterruptedException
|
|
* @throws InterruptedException
|
|
*/
|
|
*/
|
|
private void processRpcOutOfBandRequest(RpcRequestHeaderProto header,
|
|
private void processRpcOutOfBandRequest(RpcRequestHeaderProto header,
|
|
- DataInputStream dis) throws WrappedRpcServerException, IOException,
|
|
|
|
- InterruptedException {
|
|
|
|
|
|
+ RpcWritable.Buffer buffer) throws WrappedRpcServerException,
|
|
|
|
+ IOException, InterruptedException {
|
|
final int callId = header.getCallId();
|
|
final int callId = header.getCallId();
|
|
if (callId == CONNECTION_CONTEXT_CALL_ID) {
|
|
if (callId == CONNECTION_CONTEXT_CALL_ID) {
|
|
// SASL must be established prior to connection context
|
|
// SASL must be established prior to connection context
|
|
@@ -2180,7 +2172,7 @@ public abstract class Server {
|
|
"Connection header sent during SASL negotiation");
|
|
"Connection header sent during SASL negotiation");
|
|
}
|
|
}
|
|
// read and authorize the user
|
|
// read and authorize the user
|
|
- processConnectionContext(dis);
|
|
|
|
|
|
+ processConnectionContext(buffer);
|
|
} else if (callId == AuthProtocol.SASL.callId) {
|
|
} else if (callId == AuthProtocol.SASL.callId) {
|
|
// if client was switched to simple, ignore first SASL message
|
|
// if client was switched to simple, ignore first SASL message
|
|
if (authProtocol != AuthProtocol.SASL) {
|
|
if (authProtocol != AuthProtocol.SASL) {
|
|
@@ -2188,7 +2180,7 @@ public abstract class Server {
|
|
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
|
|
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
|
|
"SASL protocol not requested by client");
|
|
"SASL protocol not requested by client");
|
|
}
|
|
}
|
|
- saslReadAndProcess(dis);
|
|
|
|
|
|
+ saslReadAndProcess(buffer);
|
|
} else if (callId == PING_CALL_ID) {
|
|
} else if (callId == PING_CALL_ID) {
|
|
LOG.debug("Received ping message");
|
|
LOG.debug("Received ping message");
|
|
} else {
|
|
} else {
|
|
@@ -2235,13 +2227,12 @@ public abstract class Server {
|
|
* @throws WrappedRpcServerException - deserialization failed
|
|
* @throws WrappedRpcServerException - deserialization failed
|
|
*/
|
|
*/
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
- private <T extends Message> T decodeProtobufFromStream(Builder builder,
|
|
|
|
- DataInputStream dis) throws WrappedRpcServerException {
|
|
|
|
|
|
+ <T extends Message> T getMessage(Message message,
|
|
|
|
+ RpcWritable.Buffer buffer) throws WrappedRpcServerException {
|
|
try {
|
|
try {
|
|
- builder.mergeDelimitedFrom(dis);
|
|
|
|
- return (T)builder.build();
|
|
|
|
|
|
+ return (T)buffer.getValue(message);
|
|
} catch (Exception ioe) {
|
|
} catch (Exception ioe) {
|
|
- Class<?> protoClass = builder.getDefaultInstanceForType().getClass();
|
|
|
|
|
|
+ Class<?> protoClass = message.getClass();
|
|
throw new WrappedRpcServerException(
|
|
throw new WrappedRpcServerException(
|
|
RpcErrorCodeProto.FATAL_DESERIALIZING_REQUEST,
|
|
RpcErrorCodeProto.FATAL_DESERIALIZING_REQUEST,
|
|
"Error decoding " + protoClass.getSimpleName() + ": "+ ioe);
|
|
"Error decoding " + protoClass.getSimpleName() + ": "+ ioe);
|
|
@@ -2632,25 +2623,20 @@ public abstract class Server {
|
|
private void setupResponse(Call call,
|
|
private void setupResponse(Call call,
|
|
RpcResponseHeaderProto header, Writable rv) throws IOException {
|
|
RpcResponseHeaderProto header, Writable rv) throws IOException {
|
|
ResponseBuffer buf = responseBuffer.get().reset();
|
|
ResponseBuffer buf = responseBuffer.get().reset();
|
|
- // adjust capacity on estimated length to reduce resizing copies
|
|
|
|
- int estimatedLen = header.getSerializedSize();
|
|
|
|
- estimatedLen += CodedOutputStream.computeRawVarint32Size(estimatedLen);
|
|
|
|
- // if it's not a wrapped protobuf, just let it grow on its own
|
|
|
|
- if (rv instanceof RpcWrapper) {
|
|
|
|
- estimatedLen += ((RpcWrapper)rv).getLength();
|
|
|
|
- }
|
|
|
|
- buf.ensureCapacity(estimatedLen);
|
|
|
|
- header.writeDelimitedTo(buf);
|
|
|
|
- if (rv != null) { // null for exceptions
|
|
|
|
- rv.write(buf);
|
|
|
|
- }
|
|
|
|
- call.setResponse(ByteBuffer.wrap(buf.toByteArray()));
|
|
|
|
- // Discard a large buf and reset it back to smaller size
|
|
|
|
- // to free up heap.
|
|
|
|
- if (buf.capacity() > maxRespSize) {
|
|
|
|
- LOG.warn("Large response size " + buf.size() + " for call "
|
|
|
|
- + call.toString());
|
|
|
|
- buf.setCapacity(INITIAL_RESP_BUF_SIZE);
|
|
|
|
|
|
+ try {
|
|
|
|
+ RpcWritable.wrap(header).writeTo(buf);
|
|
|
|
+ if (rv != null) {
|
|
|
|
+ RpcWritable.wrap(rv).writeTo(buf);
|
|
|
|
+ }
|
|
|
|
+ call.setResponse(ByteBuffer.wrap(buf.toByteArray()));
|
|
|
|
+ } finally {
|
|
|
|
+ // Discard a large buf and reset it back to smaller size
|
|
|
|
+ // to free up heap.
|
|
|
|
+ if (buf.capacity() > maxRespSize) {
|
|
|
|
+ LOG.warn("Large response size " + buf.size() + " for call "
|
|
|
|
+ + call.toString());
|
|
|
|
+ buf.setCapacity(INITIAL_RESP_BUF_SIZE);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2701,7 +2687,7 @@ public abstract class Server {
|
|
.setState(SaslState.WRAP)
|
|
.setState(SaslState.WRAP)
|
|
.setToken(ByteString.copyFrom(token))
|
|
.setToken(ByteString.copyFrom(token))
|
|
.build();
|
|
.build();
|
|
- setupResponse(call, saslHeader, new RpcResponseWrapper(saslMessage));
|
|
|
|
|
|
+ setupResponse(call, saslHeader, RpcWritable.wrap(saslMessage));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|