Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDFS-17650. [ARR] The router server-side rpc protocol PB supports asynchrony. #7139

Merged
merged 12 commits into from
Jan 9, 2025
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,21 @@

import org.apache.hadoop.hdfs.server.federation.router.ThreadLocalContext;
import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction;
import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.ipc.CallerContext;
import org.apache.hadoop.ipc.Client;
import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.ProtobufRpcEngineCallback2;
import org.apache.hadoop.ipc.internal.ShadedProtobufHelper;
import org.apache.hadoop.thirdparty.protobuf.Message;
import org.apache.hadoop.util.concurrent.AsyncGet;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.Executor;

import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException;
Expand Down Expand Up @@ -96,6 +101,45 @@ public static <T, R> R asyncIpcClient(
return asyncReturn(clazz);
}

/**
* Asynchronously invokes an RPC call and applies a response transformation function
* to the result on server-side.
* @param req The IPC call encapsulating the RPC request on server-side.
* @param res The function to apply to the response of the RPC call on server-side.
* @param <T> Type of the call's result.
*/
public static <T> void asyncRouterServer(ServerReq<T> req, ServerRes<T> res) {
final ProtobufRpcEngineCallback2 callback =
ProtobufRpcEngine2.Server.registerForDeferredResponse2();

CompletableFuture<Object> completableFuture =
CompletableFuture.completedFuture(null);
completableFuture.thenCompose(o -> {
Comment on lines +111 to +117
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's better to add some comments

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done.

try {
req.req();
return (CompletableFuture<T>) AsyncUtil.getAsyncUtilCompletableFuture();
} catch (Exception e) {
throw new CompletionException(e);
}
}).handle((result, e) -> {
LOG.debug("Async response, callback: {}, CallerContext: {}, result: [{}], exception: [{}]",
callback, CallerContext.getCurrent(), result, e);
if (e == null) {
Message value = null;
try {
value = res.res(result);
} catch (Exception re) {
callback.error(re);
return null;
}
callback.setResponse(value);
} else {
callback.error(e.getCause());
}
return null;
});
}

/**
* Sets the executor used for handling responses asynchronously within
* the utility class.
Expand All @@ -105,4 +149,14 @@ public static <T, R> R asyncIpcClient(
public static void setWorker(Executor worker) {
AsyncRpcProtocolPBUtil.worker = worker;
}

@FunctionalInterface
interface ServerReq<T> {
T req() throws Exception;
}

@FunctionalInterface
interface ServerRes<T> {
Message res(T result) throws Exception;
}
}

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/**
* 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.
*/
package org.apache.hadoop.hdfs.protocolPB;

import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer;
import org.apache.hadoop.thirdparty.protobuf.RpcController;
import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.tools.GetUserMappingsProtocol;
import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto;
import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto;
import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB;

import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncRouterServer;

public class RouterGetUserMappingsProtocolServerSideTranslatorPB
extends GetUserMappingsProtocolServerSideTranslatorPB {
private final RouterRpcServer server;
private final boolean isAsyncRpc;

public RouterGetUserMappingsProtocolServerSideTranslatorPB(GetUserMappingsProtocol impl) {
super(impl);
this.server = (RouterRpcServer) impl;
this.isAsyncRpc = server.isAsync();
}

@Override
public GetGroupsForUserResponseProto getGroupsForUser(
RpcController controller,
GetGroupsForUserRequestProto request) throws ServiceException {
if (!isAsyncRpc) {
return super.getGroupsForUser(controller, request);
}
asyncRouterServer(() -> server.getGroupsForUser(request.getUser()), groups -> {
GetGroupsForUserResponseProto.Builder builder =
GetGroupsForUserResponseProto
.newBuilder();
for (String g : groups) {
builder.addGroups(g);
}
return builder.build();
});
return null;
}
}
Loading
Loading