Skip to content

Commit

Permalink
HDFS-13399. [SBN read] Make Client field AlignmentContext non-static.…
Browse files Browse the repository at this point in the history
… Contributed by Plamen Jeliazkov.
  • Loading branch information
pjeli authored and shvachko committed Dec 24, 2018
1 parent a109f2b commit e880660
Show file tree
Hide file tree
Showing 13 changed files with 619 additions and 252 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -103,12 +103,6 @@ protected Boolean initialValue() {
return false;
}
};
private static AlignmentContext alignmentContext;

/** Set alignment context to use to fetch state alignment info from RPC. */
public static void setAlignmentContext(AlignmentContext ac) {
alignmentContext = ac;
}

@SuppressWarnings("unchecked")
@Unstable
Expand Down Expand Up @@ -345,6 +339,7 @@ static class Call {
final RPC.RpcKind rpcKind; // Rpc EngineKind
boolean done; // true when call is done
private final Object externalHandler;
private AlignmentContext alignmentContext;

private Call(RPC.RpcKind rpcKind, Writable param) {
this.rpcKind = rpcKind;
Expand Down Expand Up @@ -386,6 +381,15 @@ protected synchronized void callComplete() {
}
}

/**
* Set an AlignmentContext for the call to update when call is done.
*
* @param ac alignment context to update.
*/
public synchronized void setAlignmentContext(AlignmentContext ac) {
this.alignmentContext = ac;
}

/** Set the exception when there is an error.
* Notify the caller the call is done.
*
Expand Down Expand Up @@ -1114,7 +1118,7 @@ public void sendRpcRequest(final Call call)
// Items '1' and '2' are prepared here.
RpcRequestHeaderProto header = ProtoUtil.makeRpcRequestHeader(
call.rpcKind, OperationProto.RPC_FINAL_PACKET, call.id, call.retry,
clientId, alignmentContext);
clientId, call.alignmentContext);

final ResponseBuffer buf = new ResponseBuffer();
header.writeDelimitedTo(buf);
Expand Down Expand Up @@ -1191,9 +1195,9 @@ private void receiveRpcResponse() {
Writable value = packet.newInstance(valueClass, conf);
final Call call = calls.remove(callId);
call.setRpcResponse(value);
}
if (alignmentContext != null) {
alignmentContext.receiveResponseState(header);
if (call.alignmentContext != null) {
call.alignmentContext.receiveResponseState(header);
}
}
// verify that packet length was correct
if (packet.remaining() > 0) {
Expand Down Expand Up @@ -1374,7 +1378,15 @@ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId, AtomicBoolean fallbackToSimpleAuth)
throws IOException {
return call(rpcKind, rpcRequest, remoteId, RPC.RPC_SERVICE_CLASS_DEFAULT,
fallbackToSimpleAuth);
fallbackToSimpleAuth, null);
}

public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId, AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
return call(rpcKind, rpcRequest, remoteId, RPC.RPC_SERVICE_CLASS_DEFAULT,
fallbackToSimpleAuth, alignmentContext);
}

private void checkAsyncCall() throws IOException {
Expand All @@ -1391,6 +1403,14 @@ private void checkAsyncCall() throws IOException {
}
}

Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId, int serviceClass,
AtomicBoolean fallbackToSimpleAuth)
throws IOException {
return call(rpcKind, rpcRequest, remoteId, serviceClass,
fallbackToSimpleAuth, null);
}

/**
* Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
* <code>remoteId</code>, returning the rpc response.
Expand All @@ -1401,14 +1421,17 @@ private void checkAsyncCall() throws IOException {
* @param serviceClass - service class for RPC
* @param fallbackToSimpleAuth - set to true or false during this method to
* indicate if a secure client falls back to simple auth
* @param alignmentContext - state alignment context
* @return the rpc response
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
*/
Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId, int serviceClass,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
final Call call = createCall(rpcKind, rpcRequest);
call.setAlignmentContext(alignmentContext);
final Connection connection = getConnection(remoteId, call, serviceClass,
fallbackToSimpleAuth);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,18 +86,20 @@ public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy
) throws IOException {
return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, null);
rpcTimeout, connectionRetryPolicy, null, null);
}

@Override
@SuppressWarnings("unchecked")
public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {

final Invoker invoker = new Invoker(protocol, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, fallbackToSimpleAuth);
rpcTimeout, connectionRetryPolicy, fallbackToSimpleAuth,
alignmentContext);
return new ProtocolProxy<T>(protocol, (T) Proxy.newProxyInstance(
protocol.getClassLoader(), new Class[]{protocol}, invoker), false);
}
Expand All @@ -122,15 +124,18 @@ private static class Invoker implements RpcInvocationHandler {
private final long clientProtocolVersion;
private final String protocolName;
private AtomicBoolean fallbackToSimpleAuth;
private AlignmentContext alignmentContext;

private Invoker(Class<?> protocol, InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf, SocketFactory factory,
int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
this(protocol, Client.ConnectionId.getConnectionId(
addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf),
conf, factory);
this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.alignmentContext = alignmentContext;
}

/**
Expand Down Expand Up @@ -227,7 +232,7 @@ public Message invoke(Object proxy, final Method method, Object[] args)
try {
val = (RpcWritable.Buffer) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
new RpcProtobufRequest(rpcRequestHeader, theRequest), remoteId,
fallbackToSimpleAuth);
fallbackToSimpleAuth, alignmentContext);

} catch (Throwable e) {
if (LOG.isTraceEnabled()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -586,7 +586,44 @@ public static <T> ProtocolProxy<T> getProtocolProxy(Class<T> protocol,
}
return getProtocolEngine(protocol, conf).getProxy(protocol, clientVersion,
addr, ticket, conf, factory, rpcTimeout, connectionRetryPolicy,
fallbackToSimpleAuth);
fallbackToSimpleAuth, null);
}

/**
* Get a protocol proxy that contains a proxy connection to a remote server
* and a set of methods that are supported by the server
*
* @param protocol protocol
* @param clientVersion client's version
* @param addr server address
* @param ticket security ticket
* @param conf configuration
* @param factory socket factory
* @param rpcTimeout max time for each rpc; 0 means no timeout
* @param connectionRetryPolicy retry policy
* @param fallbackToSimpleAuth set to true or false during calls to indicate
* if a secure client falls back to simple auth
* @param alignmentContext state alignment context
* @return the proxy
* @throws IOException if any error occurs
*/
public static <T> ProtocolProxy<T> getProtocolProxy(Class<T> protocol,
long clientVersion,
InetSocketAddress addr,
UserGroupInformation ticket,
Configuration conf,
SocketFactory factory,
int rpcTimeout,
RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
if (UserGroupInformation.isSecurityEnabled()) {
SaslRpcServer.init(conf);
}
return getProtocolEngine(protocol, conf).getProxy(protocol, clientVersion,
addr, ticket, conf, factory, rpcTimeout, connectionRetryPolicy,
fallbackToSimpleAuth, alignmentContext);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,8 @@ <T> ProtocolProxy<T> getProxy(Class<T> protocol,
UserGroupInformation ticket, Configuration conf,
SocketFactory factory, int rpcTimeout,
RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException;
AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext) throws IOException;

/**
* Construct a server for a protocol implementation instance.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -856,10 +856,15 @@ private class RpcCall extends Call {
final Writable rpcRequest; // Serialized Rpc request from client
ByteBuffer rpcResponse; // the response for this call

private RpcResponseHeaderProto bufferedHeader; // the response header
private Writable bufferedRv; // the byte response

RpcCall(RpcCall call) {
super(call);
this.connection = call.connection;
this.rpcRequest = call.rpcRequest;
this.bufferedRv = call.bufferedRv;
this.bufferedHeader = call.bufferedHeader;
}

RpcCall(Connection connection, int id) {
Expand All @@ -880,6 +885,14 @@ private class RpcCall extends Call {
this.rpcRequest = param;
}

public void setBufferedHeader(RpcResponseHeaderProto header) {
this.bufferedHeader = header;
}

public void setBufferedRv(Writable rv) {
this.bufferedRv = rv;
}

@Override
public String getProtocol() {
return "rpc";
Expand Down Expand Up @@ -968,6 +981,13 @@ void doResponse(Throwable t) throws IOException {
setupResponse(call,
RpcStatusProto.FATAL, RpcErrorCodeProto.ERROR_RPC_SERVER,
null, t.getClass().getName(), StringUtils.stringifyException(t));
} else if (alignmentContext != null) {
// rebuild response with state context in header
RpcResponseHeaderProto.Builder responseHeader =
call.bufferedHeader.toBuilder();
alignmentContext.updateResponseState(responseHeader);
RpcResponseHeaderProto builtHeader = responseHeader.build();
setupResponse(call, builtHeader, call.bufferedRv);
}
connection.sendResponse(call);
}
Expand Down Expand Up @@ -2992,9 +3012,6 @@ private void setupResponse(
headerBuilder.setRetryCount(call.retryCount);
headerBuilder.setStatus(status);
headerBuilder.setServerIpcVersionNum(CURRENT_VERSION);
if(alignmentContext != null) {
alignmentContext.updateResponseState(headerBuilder);
}

if (status == RpcStatusProto.SUCCESS) {
RpcResponseHeaderProto header = headerBuilder.build();
Expand All @@ -3021,6 +3038,12 @@ private void setupResponse(

private void setupResponse(RpcCall call,
RpcResponseHeaderProto header, Writable rv) throws IOException {
if (alignmentContext != null && call.bufferedHeader == null
&& call.bufferedRv == null) {
call.setBufferedHeader(header);
call.setBufferedRv(rv);
}

final byte[] response;
if (rv == null || (rv instanceof RpcWritable.ProtobufWrapper)) {
response = setupResponseForProtobuf(header, rv);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -214,16 +214,19 @@ private static class Invoker implements RpcInvocationHandler {
private Client client;
private boolean isClosed = false;
private final AtomicBoolean fallbackToSimpleAuth;
private final AlignmentContext alignmentContext;

public Invoker(Class<?> protocol,
InetSocketAddress address, UserGroupInformation ticket,
Configuration conf, SocketFactory factory,
int rpcTimeout, AtomicBoolean fallbackToSimpleAuth)
int rpcTimeout, AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
this.remoteId = Client.ConnectionId.getConnectionId(address, protocol,
ticket, rpcTimeout, null, conf);
this.client = CLIENTS.getClient(conf, factory);
this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.alignmentContext = alignmentContext;
}

@Override
Expand All @@ -246,7 +249,7 @@ public Object invoke(Object proxy, Method method, Object[] args)
try {
value = (ObjectWritable)
client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args),
remoteId, fallbackToSimpleAuth);
remoteId, fallbackToSimpleAuth, alignmentContext);
} finally {
if (traceScope != null) traceScope.close();
}
Expand Down Expand Up @@ -289,7 +292,7 @@ public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
int rpcTimeout, RetryPolicy connectionRetryPolicy)
throws IOException {
return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, null);
rpcTimeout, connectionRetryPolicy, null, null);
}

/** Construct a client-side proxy object that implements the named protocol,
Expand All @@ -301,7 +304,8 @@ public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
InetSocketAddress addr, UserGroupInformation ticket,
Configuration conf, SocketFactory factory,
int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth)
AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {

if (connectionRetryPolicy != null) {
Expand All @@ -311,7 +315,7 @@ public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,

T proxy = (T) Proxy.newProxyInstance(protocol.getClassLoader(),
new Class[] { protocol }, new Invoker(protocol, addr, ticket, conf,
factory, rpcTimeout, fallbackToSimpleAuth));
factory, rpcTimeout, fallbackToSimpleAuth, alignmentContext));
return new ProtocolProxy<T>(protocol, proxy, true);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -278,7 +278,7 @@ public <T> ProtocolProxy<T> getProxy(
SocketFactory factory, int rpcTimeout,
RetryPolicy connectionRetryPolicy) throws IOException {
return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, null);
rpcTimeout, connectionRetryPolicy, null, null);
}

@SuppressWarnings("unchecked")
Expand All @@ -287,7 +287,8 @@ public <T> ProtocolProxy<T> getProxy(
Class<T> protocol, long clientVersion, InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf, SocketFactory factory,
int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
T proxy = (T) Proxy.newProxyInstance(protocol.getClassLoader(),
new Class[] { protocol }, new StoppedInvocationHandler());
return new ProtocolProxy<T>(protocol, proxy, false);
Expand Down
Loading

0 comments on commit e880660

Please sign in to comment.