RPC :Remote Procedure Call
本地程序像调用本地方法一样调用远程计算机上的应用程序,采用客户端/服务器模式,请求程序是客户端,服务提供程序就是服务端
hadoop-common-project\hadoop-common\src\main\java\org\apache\hadoop\ipc\RPC.java
RPC类提供一个统一的接口,客户端通过getProxy()方法获取代理对象,服务端通过内部类Builder.build()方法构造server。
RPC类在namenode中的使用
获取代理对象
在org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer implements NamenodeProtocols类中调用createNameNodeProxy方法
private static Object createNameNodeProxy(InetSocketAddress address,
Configuration conf, UserGroupInformation ugi, Class<?> xface,
int rpcTimeout) throws IOException {
//设置序列化方式
RPC.setProtocolEngine(conf, xface, ProtobufRpcEngine.class);
//获取代理对象
Object proxy = RPC.getProxy(xface, RPC.getProtocolVersion(xface), address,
ugi, conf, NetUtils.getDefaultSocketFactory(conf), rpcTimeout);
return proxy;
}
构造server
在org.apache.hadoop.hdfs.server.namenode.NameNode类中,初始化方法中调用createRpcServer方法
protected void initialize(Configuration conf) throws IOException {
...
//创建rpc server
rpcServer = createRpcServer(conf);
...
}
在createRpcServer方法中创建NameNodeRpcServer对象
protected NameNodeRpcServer createRpcServer(Configuration conf)
throws IOException {
return new NameNodeRpcServer(conf, this);
}
类org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer implements NamenodeProtocols构造函数:
public NameNodeRpcServer(Configuration conf, NameNode nn)
throws IOException {
...
serviceRpcServer = new RPC.Builder(conf)
.setProtocol(
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB.class)
.setInstance(clientNNPbService)
.setBindAddress(bindHost)
.setPort(serviceRpcAddr.getPort())
.setNumHandlers(serviceHandlerCount)
.setVerbose(false)
.setSecretManager(namesystem.getDelegationTokenSecretManager())
.build();
...
}
nameNode启动rpc server:
private void startCommonServices(Configuration conf) throws IOException {
...
rpcServer.start();
...
}
hadoop-common-project\hadoop-common\src\main\java\org\apache\hadoop\ipc\Server.java
主从多线程Reactor编程模型
Server的启动:
public synchronized void start() {
responder.start();
listener.start();
handlers = new Handler[handlerCount];
for (int i = 0; i < handlerCount; i++) {
handlers[i] = new Handler(i);
handlers[i].start();
}
}
Server.Listener:负责处理OP_ACCEPT事件
Listener构造函数:
public Listener() throws IOException {
创建ServerSocketChannel,Selector
address = new InetSocketAddress(bindAddress, port);
acceptChannel = ServerSocketChannel.open();
acceptChannel.configureBlocking(false);
bind(acceptChannel.socket(), address, backlogLength, conf, portRangeConfig);
port = acceptChannel.socket().getLocalPort(); //Could be an ephemeral port
selector= Selector.open();
启动reader线程:
readers = new Reader[readThreads];
for (int i = 0; i < readThreads; i++) {
Reader reader = new Reader(
"Socket Reader #" + (i + 1) + " for port " + port);
readers[i] = reader;
reader.start();
}
线程个数通过读取配置文件的方式获得,key值为 ipc.server.read.threadpool.size:
this.readThreads = conf.getInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY,CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT);
// 默认值为1
设置listener线程监听OP_ACCEPT事件
acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
}
Listener线程:
@Override
public void run() {
...
//关闭无效的连接
connectionManager.startIdleScan();
while (running) {
SelectionKey key = null;
try {
//等待连接
getSelector().select();
Iterator<SelectionKey> iter = getSelector().selectedKeys().iterator();
while (iter.hasNext()) {
key = iter.next();
iter.remove();
try {
if (key.isValid()) {
//如果是OP_ACCEPT事件
if (key.isAcceptable())
doAccept(key);
}
} catch (IOException e) {
}
key = null;
}
}
...
}
}
doAccept方法:
void doAccept(SelectionKey key) throws InterruptedException, IOException, OutOfMemoryError {
ServerSocketChannel server = (ServerSocketChannel) key.channel();
SocketChannel channel;
while ((channel = server.accept()) != null) {
channel.configureBlocking(false);
channel.socket().setTcpNoDelay(tcpNoDelay);
channel.socket().setKeepAlive(true);
*****************************************获取reader引用*****************************************
Reader reader = getReader();
Connection c = connectionManager.register(channel);
if (c == null) {
if (channel.isOpen()) {
IOUtils.cleanup(null, channel);
}
connectionManager.droppedConnections.getAndIncrement();
continue;
}
*********************************绑定事件处理器************************************
key.attach(c);
reader.addConnection(c);
}
}
addConnection方法:
public void addConnection(Connection conn) throws InterruptedException {
pendingConnections.put(conn);
readSelector.wakeup();
}
//pendingConnections是reader类的私有成员变量
//BlockingQueue<Connection> pendingConnections;
Server.Listener.Reader:负责处理OP_READ事件
Reader构造函数:
Reader(String name) throws IOException {
super(name);
this.pendingConnections =
new LinkedBlockingQueue<Connection>(readerPendingConnectionQueue);
this.readSelector = Selector.open();
}
reader线程:
@Override
public void run() {
LOG.info("Starting " + Thread.currentThread().getName());
try
{
doRunLoop();
} finally {
try {
readSelector.close();
} catch (IOException ioe) {
LOG.error("Error closing read selector in " + Thread.currentThread().getName(), ioe);
}
}
}
doRunLoop方法:
private synchronized void doRunLoop() {
while (running) {
SelectionKey key = null;
try {
int size = pendingConnections.size();
for (int i=size; i>0; i--) {
从pendingConnections队列中取出一个connection,如果队列为空,阻塞
Connection conn = pendingConnections.take();
监听OP_READ事件
conn.channel.register(readSelector, SelectionKey.OP_READ, conn);
}
等待OP_READ事件
readSelector.select();
Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
while (iter.hasNext()) {
key = iter.next();
iter.remove();
try {
OP_READ事件处理:
if (key.isReadable()) {
doRead(key);
}
}
...
}
doRead方法:
void doRead(SelectionKey key) throws InterruptedException {
int count;
Connection c = (Connection)key.attachment();
...
try {
数据处理
count = c.readAndProcess();
}
...
}
Rpc-connection header
/**
* +----------------------------------+
* | "hrpc" 4 bytes |
* +----------------------------------+
* | Version (1 byte) |
* +----------------------------------+
* | Service Class (1 byte) |
* +----------------------------------+
* | AuthProtocol (1 byte) |
* +----------------------------------+
The format of RPC call
*
* +--------------------------------------------------------------+
* | Rpc length in bytes (4 bytes int) sum of next two parts |
* +--------------------------------------------------------------+
* | RpcRequestHeaderProto - serialized delimited ie has len |
* +--------------------------------------------------------------+
* | RpcRequest The actual rpc request |
* | This request is serialized based on RpcKindProto |
* +--------------------------------------------------------------+
*
*/
readAndProcess方法:
public int readAndProcess() throws IOException, InterruptedException {
while (!shouldClose()) { // stop if a fatal response has been sent.
rpc header处理:
// dataLengthBuffer is used to read "hrpc" or the rpc-packet length
int count = -1;
if (dataLengthBuffer.remaining() > 0) {
//从channel中读取数据到dataLengthBuffer中,返回值为读取到的字节数
count = channelRead(channel, dataLengthBuffer);
if (count < 0 || dataLengthBuffer.remaining() > 0)
return count;
}
if (!connectionHeaderRead) {
// Every connection is expected to send the header;
// so far we read "hrpc" of the connection header.
if (connectionHeaderBuf == null) {
// for the bytes that follow "hrpc", in the connection header
//public static final int HEADER_LEN_AFTER_HRPC_PART = 3;
connectionHeaderBuf = ByteBuffer.allocate(HEADER_LEN_AFTER_HRPC_PART);
}
//读取三个字节
count = channelRead(channel, connectionHeaderBuf);
if (count < 0 || connectionHeaderBuf.remaining() > 0) {
return count;
}
//获取版本信息
int version = connectionHeaderBuf.get(0);
// TODO we should add handler for service class later
//获取Service Class
this.setServiceClass(connectionHeaderBuf.get(1));
dataLengthBuffer.flip();
// Check if it looks like the user is hitting an IPC port
// with an HTTP GET - this is a common error, so we can
// send back a simple string indicating as much.
//如果向rpc端口发送http的get请求,返回错误信息
if (HTTP_GET_BYTES.equals(dataLengthBuffer)) {
setupHttpRequestOnIpcPortResponse();
return -1;
}
//校验hrpc和版本
if (!RpcConstants.HEADER.equals(dataLengthBuffer)
|| version != CURRENT_VERSION) {
//Warning is ok since this is not supposed to happen.
LOG.warn("Incorrect header or version mismatch from " +
hostAddress + ":" + remotePort +
" got version " + version +
" expected version " + CURRENT_VERSION);
setupBadVersionResponse(version);
return -1;
}
// this may switch us into SIMPLE
//获取安全协议
authProtocol = initializeAuthContext(connectionHeaderBuf.get(2));
dataLengthBuffer.clear(); // clear to next read rpc packet len
connectionHeaderBuf = null;
connectionHeaderRead = true;
continue; // connection header read, now read 4 bytes rpc packet len
}
读取rpc数据:
if (data == null) { // just read 4 bytes - length of RPC packet
dataLengthBuffer.flip();
dataLength = dataLengthBuffer.getInt();
//校验数据长度:0<dataLength<64M
//ipc.maximum.data.length
//IPC_MAXIMUM_DATA_LENGTH_DEFAULT = 64 * 1024 * 1024;
checkDataLength(dataLength);
// Set buffer for reading EXACTLY the RPC-packet length and no more.
data = ByteBuffer.allocate(dataLength);
}
// 从chanel中读取请求数据
count = channelRead(channel, data);
if (data.remaining() == 0) {
dataLengthBuffer.clear(); // to read length of future rpc packets
data.flip();
ByteBuffer requestData = data;
data = null; // null out in case processOneRpc throws.
boolean isHeaderRead = connectionContextRead;
数据处理
processOneRpc(requestData);
// the last rpc-request we processed could have simply been the
// connectionContext; if so continue to read the first RPC.
if (!isHeaderRead) {
continue;
}
}
return count;
}
return -1;
}
processOneRpc方法:
private void processOneRpc(ByteBuffer bb)
throws IOException, InterruptedException {
// exceptions that escape this method are fatal to the connection.
// setupResponse will use the rpc status to determine if the connection
// should be closed.
int callId = -1;
//默认值也是-1
int retry = RpcConstants.INVALID_RETRY_COUNT;
try {
//数据类型转换
final RpcWritable.Buffer buffer = RpcWritable.Buffer.wrap(bb);
final RpcRequestHeaderProto header =
getMessage(RpcRequestHeaderProto.getDefaultInstance(), buffer);
RpcRequestHeaderProto
message RpcRequestHeaderProto { // the header for the RpcRequest
enum OperationProto {
RPC_FINAL_PACKET = 0; // The final RPC Packet
RPC_CONTINUATION_PACKET = 1; // not implemented yet
RPC_CLOSE_CONNECTION = 2; // close the rpc connection
}
optional RpcKindProto rpcKind = 1;
optional OperationProto rpcOp = 2;
required sint32 callId = 3; // a sequence number that is sent back in response
required bytes clientId = 4; // Globally unique client ID
// clientId + callId uniquely identifies a request
// retry count, 1 means this is the first retry
optional sint32 retryCount = 5 [default = -1];
optional RPCTraceInfoProto traceInfo = 6; // tracing info
optional RPCCallerContextProto callerContext = 7; // call context
}
callId = header.getCallId();
retry = header.getRetryCount();
if (LOG.isDebugEnabled()) {
LOG.debug(" got #" + callId);
}
//校验RPC请求
// if (!header.hasRpcOp())
// if (header.getRpcOp() != RpcRequestHeaderProto.OperationProto.RPC_FINAL_PACKET)
// if (!header.hasRpcKind())
checkRpcHeaders(header);
/**
* public static final int AUTHORIZATION_FAILED_CALL_ID = -1;
* public static final int INVALID_CALL_ID = -2;
* public static final int CONNECTION_CONTEXT_CALL_ID = -3;
* public static final int PING_CALL_ID = -4;
*
*/
if (callId < 0) { // callIds typically used during connection setup
processRpcOutOfBandRequest(header, buffer);
} else if (!connectionContextRead) {
throw new FatalRpcServerException(
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER,
"Connection context not established");
} else {
处理请求:
processRpcRequest(header, buffer);
}
}
...
}
processRpcRequest方法:
private void processRpcRequest(RpcRequestHeaderProto header,
RpcWritable.Buffer buffer) throws RpcServerException,
InterruptedException {
// 获取序列化方式,getRpcRequestWrapper方法做如下操作
// RpcKindMapValue val = rpcKindMap.get(ProtoUtil.convert(rpcKind));
convert方法:
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;
}
Class<? extends Writable> rpcRequestClass =
getRpcRequestWrapper(header.getRpcKind());
if (rpcRequestClass == null) {
LOG.warn("Unknown rpc kind " + header.getRpcKind() +
" from client " + getHostAddress());
final String err = "Unknown rpc kind in rpc header" +
header.getRpcKind();
throw new FatalRpcServerException(
RpcErrorCodeProto.FATAL_INVALID_RPC_HEADER, err);
}
Writable rpcRequest;
try { //Read the rpc request
// 读取rpc请求
/**
* newInstance 利用传入的rpcRequestClass创建实例:
* instance = valueClass.newInstance();
* 然后调用getValue从byteBuffer中读取数据:
* return RpcWritable.wrap(value).readFrom(bb);
*/
rpcRequest = buffer.newInstance(rpcRequestClass, conf);
}
...
获取call属性:
TraceScope traceScope = null;
if (header.hasTraceInfo()) {
if (tracer != null) {
// If the incoming RPC included tracing info, always continue the
// trace
SpanId parentSpanId = new SpanId(
header.getTraceInfo().getTraceId(),
header.getTraceInfo().getParentId());
traceScope = tracer.newScope(
RpcClientUtil.toTraceName(rpcRequest.toString()),
parentSpanId);
traceScope.detach();
}
}
CallerContext callerContext = null;
if (header.hasCallerContext()) {
callerContext =
new CallerContext.Builder(header.getCallerContext().getContext())
.setSignature(header.getCallerContext().getSignature()
.toByteArray())
.build();
}
RpcCall call = new RpcCall(this, header.getCallId(),
header.getRetryCount(), rpcRequest,
ProtoUtil.convert(header.getRpcKind()),
header.getClientId().toByteArray(), traceScope, callerContext);
// Save the priority level assignment by the scheduler
// 设置优先级
call.setPriorityLevel(callQueue.getPriorityLevel(call));
把call对象放到callQueue队列中
try {
// 把call对象放到队列中
/**
* internalQueueCall方法调用:
* callQueue.put(call);
* callQueue:CallQueueManager实现了BlockingQueue接口
*/
internalQueueCall(call);
}
...
}
Handler线程:
@Override
public void run() {
...
while (running) {
TraceScope traceScope = null;
try {
// 从队列中取出call对象
从callQueue队列中取出call对象
final Call call = callQueue.take(); // pop the queue; maybe blocked here
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
}
//CurCall:threadLocal
CurCall.set(call);
if (call.traceScope != null) {
call.traceScope.reattach();
traceScope = call.traceScope;
traceScope.getSpan().addTimelineAnnotation("called");
}
// always update the current call context
CallerContext.setCurrent(call.callerContext);
UserGroupInformation remoteUser = call.getRemoteUser();
if (remoteUser != null) {
//安全模式,调用UGI函数
remoteUser.doAs(call);
} else {
//非安全模式
//RpcCall类继承Call类,这里调用RpcCall的run方法
call.run();
}
}
...
}
RpcCall.run方法:
@Override
public Void run() throws Exception
{
...
ResponseParams responseParams = new ResponseParams();
try {
value = call(
rpcKind, connection.protocolName, rpcRequest, timestamp);
// 发起本地调用,和序列化方式有关
hadoop-common-project\hadoop-common\src\main\java\org\apache\hadoop\ipc\ProtobufRpcEngine.java
public Writable call(RPC.Server server, String connectionProtocolName, Writable writableRequest, long receiveTime){
...
RpcProtobufRequest request = (RpcProtobufRequest) writableRequest;
RequestHeaderProto rpcRequest = request.getRequestHeader();
String methodName = rpcRequest.getMethodName();
...
}
} catch (Throwable e) {
populateResponseParamsOnError(e, responseParams);
}
//isResponseDeferred方法返回deferredResponse,deferredResponse是Call类的成员变量,初始值为false
if (!isResponseDeferred()) {
封装响应,rpc header
setupResponse(this, responseParams.returnStatus,
responseParams.detailedErr,
value, responseParams.errorClass, responseParams.error);
// 发送响应,调用Call.sendResponse方法,sendResponse中调用doResponse方法
发送响应信息
sendResponse();
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Deferring response for callId: " + this.callId);
}
}
return null;
}
RpcCall.doResponse方法:
@Override
void doResponse(Throwable t) throws IOException {
RpcCall call = this;
...
// 继续调用Connection.sendResponse方法,sendResponse中直接调用Responder.doRespond方法
connection.sendResponse(call);
}
Server.Responder
构造函数:
Responder() throws IOException {
this.setName("IPC Server Responder");
this.setDaemon(true);
writeSelector = Selector.open(); // create a selector
pending = 0;
}
Responder线程:
@Override
public void run() {
...
try {
doRunLoop();
}
...
}
doRunLoop方法:
private void doRunLoop() {
long lastPurgeTime = 0; // last check for old calls.
while (running) {
try {
waitPending(); // If a channel is being registered, wait.
//等待OP_WRITE
writeSelector.select(PURGE_INTERVAL);
Iterator<SelectionKey> iter = writeSelector.selectedKeys().iterator();
while (iter.hasNext()) {
SelectionKey key = iter.next();
iter.remove();
try {
if (key.isWritable()) {
// 写入响应
doAsyncWrite(key);
}
}
...
long now = Time.now();
//每隔十五分钟有一个清理的动作
if (now < lastPurgeTime + PURGE_INTERVAL) {
continue;
}
lastPurgeTime = now;
...
}
doAsyncWrite方法:
private void doAsyncWrite(SelectionKey key) throws IOException {
...
synchronized(call.connection.responseQueue) {
if (processResponse(call.connection.responseQueue, false)) {
try {
key.interestOps(0);
}
...
}
Responder.doRespond方法:
void doRespond(RpcCall call) throws IOException {
synchronized (call.connection.responseQueue) {
// must only wrap before adding to the responseQueue to prevent
// postponed responses from being encrypted and sent out of order.
if (call.connection.useWrap) {
wrapWithSasl(call);
}
// 将call加入LinkedList<RpcCall>队列
call.connection.responseQueue.addLast(call);
// 队列长度等于1的时候直接处理
if (call.connection.responseQueue.size() == 1) {
processResponse(call.connection.responseQueue, true);
}
}
}
processResponse方法:
private boolean processResponse(LinkedList<RpcCall> responseQueue,
boolean inHandler) throws IOException {
boolean error = true;
boolean done = false; // there is more data for this channel.
int numElements = 0;
RpcCall call = null;
try {
synchronized (responseQueue) {
numElements = responseQueue.size();
//如果队列是空的,直接返回true
if (numElements == 0) {
error = false;
return true; // no more data for this channel.
}
//从队列中取出一个call
call = responseQueue.removeFirst();
SocketChannel channel = call.connection.channel;
...
向channel中写入数据:
int numBytes = channelWrite(channel, call.rpcResponse);
...
}
Server.Connection类
构造方法
public Connection(SocketChannel channel, long lastContact) {
this.channel = channel;
this.lastContact = lastContact;
this.data = null;
//“hrpc” rpc数据头
// the buffer is initialized to read the "hrpc" and after that to read
// the length of the Rpc-packet (i.e 4 bytes)
this.dataLengthBuffer = ByteBuffer.allocate(4);
this.unwrappedData = null;
this.unwrappedDataLengthBuffer = ByteBuffer.allocate(4);
this.socket = channel.socket();
this.addr = socket.getInetAddress();
if (addr == null) {
this.hostAddress = "*Unknown*";
} else {
this.hostAddress = addr.getHostAddress();
}
this.remotePort = socket.getPort();
this.responseQueue = new LinkedList<RpcCall>();
if (socketSendBufferSize != 0) {
try {
socket.setSendBufferSize(socketSendBufferSize);
} catch (IOException e) {
LOG.warn("Connection: unable to set socket send buffer size to " +
socketSendBufferSize);
}
}
}