versionMap = getVersionSignatureMap(
serverAddress, protocol.getName(), rpcKind.toString());
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
deleted file mode 100644
index 6e97159fb46..00000000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
+++ /dev/null
@@ -1,118 +0,0 @@
-package org.apache.hadoop.ipc;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * This is the rpc payload header. It is sent with every rpc call
- *
- * The format of RPC call is as follows:
- * +---------------------------------------------------+
- * | Rpc length in bytes (header + payload length) |
- * +---------------------------------------------------+
- * | Rpc Header | Rpc Payload |
- * +---------------------------------------------------+
- *
- * The format of Rpc Header is:
- * +----------------------------------+
- * | RpcKind (1 bytes) |
- * +----------------------------------+
- * | RpcPayloadOperation (1 bytes) |
- * +----------------------------------+
- * | Call ID (4 bytes) |
- * +----------------------------------+
- *
- * {@link RpcKind} determines the type of serialization used for Rpc Payload.
- *
- *
- * Note this header does NOT have its own version number,
- * it used the version number from the connection header.
- */
-public class RpcPayloadHeader implements Writable {
- public enum RpcPayloadOperation {
- RPC_FINAL_PAYLOAD ((short)1),
- RPC_CONTINUATION_PAYLOAD ((short)2), // not implemented yet
- RPC_CLOSE_CONNECTION ((short)3); // close the rpc connection
-
- private final short code;
- private static final short FIRST_INDEX = RPC_FINAL_PAYLOAD.code;
- RpcPayloadOperation(short val) {
- this.code = val;
- }
-
- public void write(DataOutput out) throws IOException {
- out.writeByte(code);
- }
-
- static RpcPayloadOperation readFields(DataInput in) throws IOException {
- short inValue = in.readByte();
- return RpcPayloadOperation.values()[inValue - FIRST_INDEX];
- }
- }
-
- public enum RpcKind {
- RPC_BUILTIN ((short) 1), // Used for built in calls by tests
- RPC_WRITABLE ((short) 2), // Use WritableRpcEngine
- RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
- final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
- private static final short FIRST_INDEX = RPC_BUILTIN.value;
- private final short value;
-
- RpcKind(short val) {
- this.value = val;
- }
-
- public void write(DataOutput out) throws IOException {
- out.writeByte(value);
- }
-
- static RpcKind readFields(DataInput in) throws IOException {
- short inValue = in.readByte();
- return RpcKind.values()[inValue - FIRST_INDEX];
- }
- }
-
- private RpcKind kind;
- private RpcPayloadOperation operation;
- private int callId;
-
- public RpcPayloadHeader() {
- kind = RpcKind.RPC_WRITABLE;
- operation = RpcPayloadOperation.RPC_CLOSE_CONNECTION;
- }
-
- public RpcPayloadHeader(RpcKind kind, RpcPayloadOperation op, int callId) {
- this.kind = kind;
- this.operation = op;
- this.callId = callId;
- }
-
- int getCallId() {
- return callId;
- }
-
- RpcKind getkind() {
- return kind;
- }
-
- RpcPayloadOperation getOperation() {
- return operation;
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- kind.write(out);
- operation.write(out);
- out.writeInt(callId);
- }
-
- @Override
- public void readFields(DataInput in) throws IOException {
- kind = RpcKind.readFields(in);
- operation = RpcPayloadOperation.readFields(in);
- this.callId = in.readInt();
- }
-}
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index 8628bb0c485..24c2f7beaa7 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -72,11 +72,10 @@ import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.ipc.RPC.RpcInvoker;
import org.apache.hadoop.ipc.RPC.VersionMismatch;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcPayloadOperation;
import org.apache.hadoop.ipc.metrics.RpcDetailedMetrics;
import org.apache.hadoop.ipc.metrics.RpcMetrics;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
+import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SaslRpcServer;
@@ -170,8 +169,8 @@ public abstract class Server {
this.rpcRequestWrapperClass = rpcRequestWrapperClass;
}
}
- static Map rpcKindMap = new
- HashMap(4);
+ static Map rpcKindMap = new
+ HashMap(4);
@@ -185,7 +184,7 @@ public abstract class Server {
* @param rpcInvoker - use to process the calls on SS.
*/
- public static void registerProtocolEngine(RpcKind rpcKind,
+ public static void registerProtocolEngine(RPC.RpcKind rpcKind,
Class extends Writable> rpcRequestWrapperClass,
RpcInvoker rpcInvoker) {
RpcKindMapValue old =
@@ -201,14 +200,14 @@ public abstract class Server {
}
public Class extends Writable> getRpcRequestWrapper(
- RpcKind rpcKind) {
+ RpcKindProto rpcKind) {
if (rpcRequestClass != null)
return rpcRequestClass;
- RpcKindMapValue val = rpcKindMap.get(rpcKind);
+ RpcKindMapValue val = rpcKindMap.get(ProtoUtil.convert(rpcKind));
return (val == null) ? null : val.rpcRequestWrapperClass;
}
- public static RpcInvoker getRpcInvoker(RpcKind rpcKind) {
+ public static RpcInvoker getRpcInvoker(RPC.RpcKind rpcKind) {
RpcKindMapValue val = rpcKindMap.get(rpcKind);
return (val == null) ? null : val.rpcInvoker;
}
@@ -403,12 +402,12 @@ public abstract class Server {
private long timestamp; // time received when response is null
// time served when response is not null
private ByteBuffer rpcResponse; // the response for this call
- private final RpcKind rpcKind;
+ private final RPC.RpcKind rpcKind;
public Call(int id, Writable param, Connection connection) {
- this( id, param, connection, RpcKind.RPC_BUILTIN );
+ this( id, param, connection, RPC.RpcKind.RPC_BUILTIN );
}
- public Call(int id, Writable param, Connection connection, RpcKind kind) {
+ public Call(int id, Writable param, Connection connection, RPC.RpcKind kind) {
this.callId = id;
this.rpcRequest = param;
this.connection = connection;
@@ -1366,7 +1365,6 @@ public abstract class Server {
if (data == null) {
dataLengthBuffer.flip();
dataLength = dataLengthBuffer.getInt();
-
if ((dataLength == Client.PING_CALL_ID) && (!useWrap)) {
// covers the !useSasl too
dataLengthBuffer.clear();
@@ -1555,22 +1553,27 @@ public abstract class Server {
private void processData(byte[] buf) throws IOException, InterruptedException {
DataInputStream dis =
new DataInputStream(new ByteArrayInputStream(buf));
- RpcPayloadHeader header = new RpcPayloadHeader();
- header.readFields(dis); // Read the RpcPayload header
+ RpcPayloadHeaderProto header = RpcPayloadHeaderProto.parseDelimitedFrom(dis);
if (LOG.isDebugEnabled())
LOG.debug(" got #" + header.getCallId());
- if (header.getOperation() != RpcPayloadOperation.RPC_FINAL_PAYLOAD) {
+ if (!header.hasRpcOp()) {
+ throw new IOException(" IPC Server: No rpc op in rpcPayloadHeader");
+ }
+ if (header.getRpcOp() != RpcPayloadOperationProto.RPC_FINAL_PAYLOAD) {
throw new IOException("IPC Server does not implement operation" +
- header.getOperation());
+ header.getRpcOp());
}
// If we know the rpc kind, get its class so that we can deserialize
// (Note it would make more sense to have the handler deserialize but
// we continue with this original design.
+ if (!header.hasRpcKind()) {
+ throw new IOException(" IPC Server: No rpc kind in rpcPayloadHeader");
+ }
Class extends Writable> rpcRequestClass =
- getRpcRequestWrapper(header.getkind());
+ getRpcRequestWrapper(header.getRpcKind());
if (rpcRequestClass == null) {
- LOG.warn("Unknown rpc kind " + header.getkind() +
+ LOG.warn("Unknown rpc kind " + header.getRpcKind() +
" from client " + getHostAddress());
final Call readParamsFailedCall =
new Call(header.getCallId(), null, this);
@@ -1578,7 +1581,7 @@ public abstract class Server {
setupResponse(responseBuffer, readParamsFailedCall, Status.FATAL, null,
IOException.class.getName(),
- "Unknown rpc kind " + header.getkind());
+ "Unknown rpc kind " + header.getRpcKind());
responder.doRespond(readParamsFailedCall);
return;
}
@@ -1589,7 +1592,7 @@ public abstract class Server {
} catch (Throwable t) {
LOG.warn("Unable to read call parameters for client " +
getHostAddress() + "on connection protocol " +
- this.protocolName + " for rpcKind " + header.getkind(), t);
+ this.protocolName + " for rpcKind " + header.getRpcKind(), t);
final Call readParamsFailedCall =
new Call(header.getCallId(), null, this);
ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream();
@@ -1601,7 +1604,8 @@ public abstract class Server {
return;
}
- Call call = new Call(header.getCallId(), rpcRequest, this, header.getkind());
+ Call call = new Call(header.getCallId(), rpcRequest, this,
+ ProtoUtil.convert(header.getRpcKind()));
callQueue.put(call); // queue the call; maybe blocked here
incRpcCount(); // Increment the rpc count
}
@@ -1991,11 +1995,11 @@ public abstract class Server {
*/
@Deprecated
public Writable call(Writable param, long receiveTime) throws Exception {
- return call(RpcKind.RPC_BUILTIN, null, param, receiveTime);
+ return call(RPC.RpcKind.RPC_BUILTIN, null, param, receiveTime);
}
/** Called for each call. */
- public abstract Writable call(RpcKind rpcKind, String protocol,
+ public abstract Writable call(RPC.RpcKind rpcKind, String protocol,
Writable param, long receiveTime) throws Exception;
/**
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
index 9104f6ff834..6fd800cad6e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
@@ -33,7 +33,6 @@ import org.apache.commons.logging.*;
import org.apache.hadoop.io.*;
import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.RPC.RpcInvoker;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
@@ -75,7 +74,7 @@ public class WritableRpcEngine implements RpcEngine {
* Register the rpcRequest deserializer for WritableRpcEngine
*/
private static synchronized void initialize() {
- org.apache.hadoop.ipc.Server.registerProtocolEngine(RpcKind.RPC_WRITABLE,
+ org.apache.hadoop.ipc.Server.registerProtocolEngine(RPC.RpcKind.RPC_WRITABLE,
Invocation.class, new Server.WritableRpcInvoker());
isInitialized = true;
}
@@ -223,7 +222,7 @@ public class WritableRpcEngine implements RpcEngine {
}
ObjectWritable value = (ObjectWritable)
- client.call(RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
+ client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
if (LOG.isDebugEnabled()) {
long callTime = System.currentTimeMillis() - startTime;
LOG.debug("Call: " + method.getName() + " " + callTime);
@@ -412,12 +411,12 @@ public class WritableRpcEngine implements RpcEngine {
protocolImpl.getClass());
}
// register protocol class and its super interfaces
- registerProtocolAndImpl(RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
+ registerProtocolAndImpl(RPC.RpcKind.RPC_WRITABLE, protocolClass, protocolImpl);
protocols = RPC.getProtocolInterfaces(protocolClass);
}
for (Class> p : protocols) {
if (!p.equals(VersionedProtocol.class)) {
- registerProtocolAndImpl(RpcKind.RPC_WRITABLE, p, protocolImpl);
+ registerProtocolAndImpl(RPC.RpcKind.RPC_WRITABLE, p, protocolImpl);
}
}
@@ -461,7 +460,7 @@ public class WritableRpcEngine implements RpcEngine {
// registered directly.
// Send the call to the highest protocol version
VerProtocolImpl highest = server.getHighestSupportedProtocol(
- RpcKind.RPC_WRITABLE, protocolName);
+ RPC.RpcKind.RPC_WRITABLE, protocolName);
if (highest == null) {
throw new IOException("Unknown protocol: " + protocolName);
}
@@ -473,10 +472,10 @@ public class WritableRpcEngine implements RpcEngine {
ProtoNameVer pv =
new ProtoNameVer(call.declaringClassProtocolName, clientVersion);
protocolImpl =
- server.getProtocolImplMap(RpcKind.RPC_WRITABLE).get(pv);
+ server.getProtocolImplMap(RPC.RpcKind.RPC_WRITABLE).get(pv);
if (protocolImpl == null) { // no match for Protocol AND Version
VerProtocolImpl highest =
- server.getHighestSupportedProtocol(RpcKind.RPC_WRITABLE,
+ server.getHighestSupportedProtocol(RPC.RpcKind.RPC_WRITABLE,
protoName);
if (highest == null) {
throw new IOException("Unknown protocol: " + protoName);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
index 3ee306b6290..0618f0631c8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
@@ -21,8 +21,10 @@ package org.apache.hadoop.util;
import java.io.DataInput;
import java.io.IOException;
+import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformationProto;
+import org.apache.hadoop.ipc.protobuf.RpcPayloadHeaderProtos.*;
import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
import org.apache.hadoop.security.UserGroupInformation;
@@ -135,4 +137,30 @@ public abstract class ProtoUtil {
}
return ugi;
}
+
+ static RpcKindProto convert(RPC.RpcKind kind) {
+ switch (kind) {
+ case RPC_BUILTIN: return RpcKindProto.RPC_BUILTIN;
+ case RPC_WRITABLE: return RpcKindProto.RPC_WRITABLE;
+ case RPC_PROTOCOL_BUFFER: return RpcKindProto.RPC_PROTOCOL_BUFFER;
+ }
+ return null;
+ }
+
+
+ public static RPC.RpcKind convert( RpcKindProto kind) {
+ switch (kind) {
+ case RPC_BUILTIN: return RPC.RpcKind.RPC_BUILTIN;
+ case RPC_WRITABLE: return RPC.RpcKind.RPC_WRITABLE;
+ case RPC_PROTOCOL_BUFFER: return RPC.RpcKind.RPC_PROTOCOL_BUFFER;
+ }
+ return null;
+ }
+
+ public static RpcPayloadHeaderProto makeRpcPayloadHeader(RPC.RpcKind rpcKind,
+ RpcPayloadOperationProto operation, int callId) {
+ RpcPayloadHeaderProto.Builder result = RpcPayloadHeaderProto.newBuilder();
+ result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId);
+ return result.build();
+ }
}
diff --git a/hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto b/hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto
new file mode 100644
index 00000000000..42dea3bde3e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/proto/RpcPayloadHeader.proto
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+option java_package = "org.apache.hadoop.ipc.protobuf";
+option java_outer_classname = "RpcPayloadHeaderProtos";
+option java_generate_equals_and_hash = true;
+
+
+/**
+ * This is the rpc payload header. It is sent with every rpc call.
+ *
+ * The format of RPC call is as follows:
+ * +-----------------------------------------------------+
+ * | Rpc length in bytes |
+ * +-----------------------------------------------------+
+ * | RpcPayloadHeader - serialized delimited ie has len |
+ * +-----------------------------------------------------+
+ * | RpcRequest Payload |
+ * +-----------------------------------------------------+
+ *
+ */
+
+
+
+/**
+ * RpcKind determine the rpcEngine and the serialization of the rpc payload
+ */
+enum RpcKindProto {
+ RPC_BUILTIN = 0; // Used for built in calls by tests
+ RPC_WRITABLE = 1; // Use WritableRpcEngine
+ RPC_PROTOCOL_BUFFER = 2; // Use ProtobufRpcEngine
+}
+
+enum RpcPayloadOperationProto {
+ RPC_FINAL_PAYLOAD = 0; // The final payload
+ RPC_CONTINUATION_PAYLOAD = 1; // not implemented yet
+ RPC_CLOSE_CONNECTION = 2; // close the rpc connection
+}
+
+message RpcPayloadHeaderProto { // the header for the RpcRequest
+ optional RpcKindProto rpcKind = 1;
+ optional RpcPayloadOperationProto rpcOp = 2;
+ optional uint32 callId = 3; // each rpc has a callId that is also used in response
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
index efb2dc1126d..5797bb524bc 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPC.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.net.NetUtils;
@@ -99,7 +98,7 @@ public class TestIPC {
}
@Override
- public Writable call(RpcKind rpcKind, String protocol, Writable param,
+ public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
long receiveTime) throws IOException {
if (sleep) {
// sleep a bit
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
index 5675cbfddf9..5f5cc1bcd32 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestIPCServerResponder.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.net.NetUtils;
/**
@@ -73,7 +72,7 @@ public class TestIPCServerResponder extends TestCase {
}
@Override
- public Writable call(RpcKind rpcKind, String protocol, Writable param,
+ public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
long receiveTime) throws IOException {
if (sleep) {
try {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
index f5acd93eb2f..0446b425087 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestMultipleProtocolServer.java
@@ -23,7 +23,6 @@ import java.net.InetSocketAddress;
import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.TestProtoBufRpc.PBServerImpl;
import org.apache.hadoop.ipc.TestProtoBufRpc.TestRpcService;
import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos.TestProtobufRpcProto;
@@ -178,9 +177,9 @@ public class TestMultipleProtocolServer {
// create a server with two handlers
server = RPC.getServer(Foo0.class,
new Foo0Impl(), ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
- server.addProtocol(RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
- server.addProtocol(RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
+ server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Foo1.class, new Foo1Impl());
+ server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Bar.class, new BarImpl());
+ server.addProtocol(RPC.RpcKind.RPC_WRITABLE, Mixin.class, new BarImpl());
// Add Protobuf server
@@ -189,7 +188,7 @@ public class TestMultipleProtocolServer {
new PBServerImpl();
BlockingService service = TestProtobufRpcProto
.newReflectiveBlockingService(pbServerImpl);
- server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
+ server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService.class,
service);
server.start();
addr = NetUtils.getConnectAddress(server);
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
index 3b9140afc4c..9e7b2694411 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestProtoBufRpc.java
@@ -24,7 +24,6 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto;
import org.apache.hadoop.ipc.protobuf.TestProtos.EchoResponseProto;
import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto;
@@ -122,7 +121,7 @@ public class TestProtoBufRpc {
BlockingService service2 = TestProtobufRpc2Proto
.newReflectiveBlockingService(server2Impl);
- server.addProtocol(RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
+ server.addProtocol(RPC.RpcKind.RPC_PROTOCOL_BUFFER, TestRpcService2.class,
service2);
server.start();
}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
index aca33ef25b7..50ae210ea9e 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPCCompatibility.java
@@ -31,7 +31,6 @@ import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureRequestProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.GetProtocolSignatureResponseProto;
import org.apache.hadoop.ipc.protobuf.ProtocolInfoProtos.ProtocolSignatureProto;
@@ -134,7 +133,7 @@ public class TestRPCCompatibility {
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class,
impl, ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+ server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@@ -201,7 +200,7 @@ System.out.println("echo int is NOT supported");
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class,
impl, ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+ server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@@ -222,7 +221,7 @@ System.out.println("echo int is NOT supported");
TestImpl2 impl = new TestImpl2();
server = RPC.getServer(TestProtocol2.class,
impl, ADDRESS, 0, 2, false, conf, null);
- server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+ server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
addr = NetUtils.getConnectAddress(server);
@@ -316,11 +315,11 @@ System.out.println("echo int is NOT supported");
TestProtocol2 proxy = RPC.getProxy(TestProtocol2.class,
TestProtocol2.versionID, addr, conf);
boolean supported = RpcClientUtil.isMethodSupported(proxy,
- TestProtocol2.class, RpcKind.RPC_WRITABLE,
+ TestProtocol2.class, RPC.RpcKind.RPC_WRITABLE,
RPC.getProtocolVersion(TestProtocol2.class), "echo");
Assert.assertTrue(supported);
supported = RpcClientUtil.isMethodSupported(proxy,
- TestProtocol2.class, RpcKind.RPC_PROTOCOL_BUFFER,
+ TestProtocol2.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
RPC.getProtocolVersion(TestProtocol2.class), "echo");
Assert.assertFalse(supported);
}
@@ -334,7 +333,7 @@ System.out.println("echo int is NOT supported");
TestImpl1 impl = new TestImpl1();
server = RPC.getServer(TestProtocol1.class, impl, ADDRESS, 0, 2, false,
conf, null);
- server.addProtocol(RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
+ server.addProtocol(RPC.RpcKind.RPC_WRITABLE, TestProtocol0.class, impl);
server.start();
ProtocolMetaInfoServerSideTranslatorPB xlator =
@@ -343,13 +342,13 @@ System.out.println("echo int is NOT supported");
GetProtocolSignatureResponseProto resp = xlator.getProtocolSignature(
null,
createGetProtocolSigRequestProto(TestProtocol1.class,
- RpcKind.RPC_PROTOCOL_BUFFER));
+ RPC.RpcKind.RPC_PROTOCOL_BUFFER));
//No signatures should be found
Assert.assertEquals(0, resp.getProtocolSignatureCount());
resp = xlator.getProtocolSignature(
null,
createGetProtocolSigRequestProto(TestProtocol1.class,
- RpcKind.RPC_WRITABLE));
+ RPC.RpcKind.RPC_WRITABLE));
Assert.assertEquals(1, resp.getProtocolSignatureCount());
ProtocolSignatureProto sig = resp.getProtocolSignatureList().get(0);
Assert.assertEquals(TestProtocol1.versionID, sig.getVersion());
@@ -366,7 +365,7 @@ System.out.println("echo int is NOT supported");
}
private GetProtocolSignatureRequestProto createGetProtocolSigRequestProto(
- Class> protocol, RpcKind rpcKind) {
+ Class> protocol, RPC.RpcKind rpcKind) {
GetProtocolSignatureRequestProto.Builder builder =
GetProtocolSignatureRequestProto.newBuilder();
builder.setProtocol(protocol.getName());