diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index 65fe89b30fc7b..f92e7410f24bf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -96,6 +96,8 @@ public class Client implements AutoCloseable { private static final ThreadLocal retryCount = new ThreadLocal(); private static final ThreadLocal EXTERNAL_CALL_HANDLER = new ThreadLocal<>(); + public static final ThreadLocal> COMPLETABLE_FUTURE_THREAD_LOCAL + = new ThreadLocal<>(); private static final ThreadLocal> ASYNC_RPC_RESPONSE = new ThreadLocal<>(); private static final ThreadLocal asynchronousMode = @@ -283,6 +285,7 @@ static class Call { boolean done; // true when call is done private final Object externalHandler; private AlignmentContext alignmentContext; + private CompletableFuture completableFuture; private Call(RPC.RpcKind rpcKind, Writable param) { this.rpcKind = rpcKind; @@ -304,6 +307,7 @@ private Call(RPC.RpcKind rpcKind, Writable param) { } this.externalHandler = EXTERNAL_CALL_HANDLER.get(); + this.completableFuture = COMPLETABLE_FUTURE_THREAD_LOCAL.get(); } @Override @@ -322,6 +326,9 @@ protected synchronized void callComplete() { externalHandler.notify(); } } + if (completableFuture != null) { + completableFuture.complete(this); + } } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index df0f734d08016..4caf0919694b5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -384,9 +384,6 @@ public RPC.Server getServer(Class protocol, Object protocolImpl, */ public static class Server extends ProtobufRpcEngine2.Server { - static final ThreadLocal currentCallback = - new ThreadLocal<>(); - static class ProtobufRpcEngineCallbackImpl implements ProtobufRpcEngineCallback { @@ -397,32 +394,33 @@ static class ProtobufRpcEngineCallbackImpl public ProtobufRpcEngineCallbackImpl() { this.server = CURRENT_CALL_INFO.get().getServer(); - this.call = Server.getCurCall().get(); this.methodName = CURRENT_CALL_INFO.get().getMethodName(); - this.setupTime = Time.now(); + this.call = Server.getCurCall().get(); + this.call.deferResponse(); + this.setupTime = Time.monotonicNowNanos(); } @Override public void setResponse(Message message) { - long processingTime = Time.now() - setupTime; - call.setDeferredResponse(RpcWritable.wrap(message)); + long processingTime = + Time.monotonicNow() - setupTime / Time.NANOSECONDS_PER_MILLISECOND; + call.setDeferredResponse(RpcWritable.wrap(message), setupTime); server.updateDeferredMetrics(methodName, processingTime); } @Override public void error(Throwable t) { - long processingTime = Time.now() - setupTime; + long processingTime = + Time.monotonicNow() - setupTime / Time.NANOSECONDS_PER_MILLISECOND; String detailedMetricsName = t.getClass().getSimpleName(); server.updateDeferredMetrics(detailedMetricsName, processingTime); - call.setDeferredError(t); + call.setDeferredError(t, setupTime); } } @InterfaceStability.Unstable public static ProtobufRpcEngineCallback registerForDeferredResponse() { - ProtobufRpcEngineCallback callback = new ProtobufRpcEngineCallbackImpl(); - currentCallback.set(callback); - return callback; + return new ProtobufRpcEngineCallbackImpl(); } /** @@ -482,11 +480,7 @@ static RpcWritable processCall(RPC.Server server, CURRENT_CALL_INFO.set(new CallInfo(server, methodName)); currentCall.setDetailedMetricsName(methodName); result = service.callBlockingMethod(methodDescriptor, null, param); - // Check if this needs to be a deferred response, - // by checking the ThreadLocal callback being set - if (currentCallback.get() != null) { - currentCall.deferResponse(); - currentCallback.set(null); + if (currentCall.isResponseDeferred()) { return null; } } catch (ServiceException e) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java index bedecc8851d6a..2080437d909fe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java @@ -396,9 +396,6 @@ public static void clearClientCache() { public static class Server extends RPC.Server { - static final ThreadLocal CURRENT_CALLBACK = - new ThreadLocal<>(); - static final ThreadLocal CURRENT_CALL_INFO = new ThreadLocal<>(); static class CallInfo { @@ -429,32 +426,44 @@ static class ProtobufRpcEngineCallbackImpl ProtobufRpcEngineCallbackImpl() { this.server = CURRENT_CALL_INFO.get().getServer(); - this.call = Server.getCurCall().get(); this.methodName = CURRENT_CALL_INFO.get().getMethodName(); - this.setupTime = Time.now(); + this.call = Server.getCurCall().get(); + this.call.deferResponse(); + this.setupTime = Time.monotonicNowNanos(); } @Override public void setResponse(Message message) { - long processingTime = Time.now() - setupTime; - call.setDeferredResponse(RpcWritable.wrap(message)); + long processingTime = + Time.monotonicNow() - setupTime / Time.NANOSECONDS_PER_MILLISECOND; + call.setDeferredResponse(RpcWritable.wrap(message), setupTime); server.updateDeferredMetrics(methodName, processingTime); } @Override public void error(Throwable t) { - long processingTime = Time.now() - setupTime; + long processingTime = + Time.monotonicNow() - setupTime / Time.NANOSECONDS_PER_MILLISECOND; String detailedMetricsName = t.getClass().getSimpleName(); server.updateDeferredMetrics(detailedMetricsName, processingTime); - call.setDeferredError(t); + call.setDeferredError(t, setupTime); + LOG.info("zjtest {} error:", this.toString(), t); + } + + @Override + public String toString() { + return "ProtobufRpcEngineCallbackImpl{" + + "server=" + server + + ", call=" + call + + ", methodName='" + methodName + '\'' + + ", setupTime=" + setupTime + + '}'; } } @InterfaceStability.Unstable public static ProtobufRpcEngineCallback2 registerForDeferredResponse2() { - ProtobufRpcEngineCallback2 callback = new ProtobufRpcEngineCallbackImpl(); - CURRENT_CALLBACK.set(callback); - return callback; + return new ProtobufRpcEngineCallbackImpl(); } /** @@ -619,11 +628,7 @@ private RpcWritable call(RPC.Server server, CURRENT_CALL_INFO.set(new CallInfo(server, methodName)); currentCall.setDetailedMetricsName(methodName); result = service.callBlockingMethod(methodDescriptor, null, param); - // Check if this needs to be a deferred response, - // by checking the ThreadLocal callback being set - if (CURRENT_CALLBACK.get() != null) { - currentCall.deferResponse(); - CURRENT_CALLBACK.set(null); + if (currentCall.isResponseDeferred()) { return null; } } catch (ServiceException e) { 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 a49b5ec692b4d..88fe8fa99ba53 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 @@ -1157,10 +1157,10 @@ public boolean isResponseDeferred() { return this.deferredResponse; } - public void setDeferredResponse(Writable response) { + public void setDeferredResponse(Writable response, long setupTime) { } - public void setDeferredError(Throwable t) { + public void setDeferredError(Throwable t, long setupTime) { } public long getTimestampNanos() { @@ -1252,24 +1252,30 @@ public Void run() throws Exception { populateResponseParamsOnError(e, responseParams); } if (!isResponseDeferred()) { - long deltaNanos = Time.monotonicNowNanos() - startNanos; - ProcessingDetails details = getProcessingDetails(); - - details.set(Timing.PROCESSING, deltaNanos, TimeUnit.NANOSECONDS); - deltaNanos -= details.get(Timing.LOCKWAIT, TimeUnit.NANOSECONDS); - deltaNanos -= details.get(Timing.LOCKSHARED, TimeUnit.NANOSECONDS); - deltaNanos -= details.get(Timing.LOCKEXCLUSIVE, TimeUnit.NANOSECONDS); - details.set(Timing.LOCKFREE, deltaNanos, TimeUnit.NANOSECONDS); - - setResponseFields(value, responseParams); - sendResponse(); - details.setReturnStatus(responseParams.returnStatus); + sendResponse(value, responseParams, startNanos); } else { LOG.debug("Deferring response for callId: {}", this.callId); } return null; } + private void sendResponse( + Writable response, + ResponseParams responseParams, long startNanos) throws IOException { + long deltaNanos = Time.monotonicNowNanos() - startNanos; + ProcessingDetails details = getProcessingDetails(); + + details.set(Timing.PROCESSING, deltaNanos, TimeUnit.NANOSECONDS); + deltaNanos -= details.get(Timing.LOCKWAIT, TimeUnit.NANOSECONDS); + deltaNanos -= details.get(Timing.LOCKSHARED, TimeUnit.NANOSECONDS); + deltaNanos -= details.get(Timing.LOCKEXCLUSIVE, TimeUnit.NANOSECONDS); + details.set(Timing.LOCKFREE, deltaNanos, TimeUnit.NANOSECONDS); + + setResponseFields(response, responseParams); + sendResponse(); + details.setReturnStatus(responseParams.returnStatus); + } + /** * @param t the {@link java.lang.Throwable} to use to set * errorInfo @@ -1326,29 +1332,11 @@ void doResponse(Throwable t, RpcStatusProto status) throws IOException { connection.sendResponse(call); } - /** - * Send a deferred response, ignoring errors. - */ - private void sendDeferedResponse() { - try { - connection.sendResponse(this); - } catch (Exception e) { - // For synchronous calls, application code is done once it's returned - // from a method. It does not expect to receive an error. - // This is equivalent to what happens in synchronous calls when the - // Responder is not able to send out the response. - LOG.error("Failed to send deferred response. ThreadName=" + Thread - .currentThread().getName() + ", CallId=" - + callId + ", hostname=" + getHostAddress()); - } - } - @Override - public void setDeferredResponse(Writable response) { + public void setDeferredResponse(Writable response, long setupTime) { if (this.connection.getServer().running) { try { - setupResponse(this, RpcStatusProto.SUCCESS, null, response, - null, null); + sendResponse(response, new ResponseParams(), setupTime); } catch (IOException e) { // For synchronous calls, application code is done once it has // returned from a method. It does not expect to receive an error. @@ -1357,14 +1345,20 @@ public void setDeferredResponse(Writable response) { LOG.error( "Failed to setup deferred successful response. ThreadName=" + Thread.currentThread().getName() + ", Call=" + this); - return; + } catch (Exception e) { + // For synchronous calls, application code is done once it's returned + // from a method. It does not expect to receive an error. + // This is equivalent to what happens in synchronous calls when the + // Responder is not able to send out the response. + LOG.error("Failed to send deferred response. ThreadName=" + Thread + .currentThread().getName() + ", CallId=" + + callId + ", hostname=" + getHostAddress()); } - sendDeferedResponse(); } } @Override - public void setDeferredError(Throwable t) { + public void setDeferredError(Throwable t, long setupTime) { if (this.connection.getServer().running) { if (t == null) { t = new IOException( @@ -1373,9 +1367,7 @@ public void setDeferredError(Throwable t) { try { ResponseParams responseParams = new ResponseParams(); populateResponseParamsOnError(t, responseParams); - setupResponse(this, responseParams.returnStatus, - responseParams.detailedErr, - null, responseParams.errorClass, responseParams.error); + sendResponse(null, responseParams, setupTime); } catch (IOException e) { // For synchronous calls, application code is done once it has // returned from a method. It does not expect to receive an error. @@ -1384,8 +1376,15 @@ public void setDeferredError(Throwable t) { LOG.error( "Failed to setup deferred error response. ThreadName=" + Thread.currentThread().getName() + ", Call=" + this); + } catch (Exception e) { + // For synchronous calls, application code is done once it's returned + // from a method. It does not expect to receive an error. + // This is equivalent to what happens in synchronous calls when the + // Responder is not able to send out the response. + LOG.error("Failed to send deferred response. ThreadName=" + Thread + .currentThread().getName() + ", CallId=" + + callId + ", hostname=" + getHostAddress()); } - sendDeferedResponse(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java index cb80d067bb8bc..fe9b3056c7544 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java @@ -35,14 +35,14 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements ProtocolMetaInterface, RefreshUserMappingsProtocol, Closeable { /** RpcController is not used and hence is set to null */ - private final static RpcController NULL_CONTROLLER = null; + protected final static RpcController NULL_CONTROLLER = null; private final RefreshUserMappingsProtocolPB rpcProxy; - private final static RefreshUserToGroupsMappingsRequestProto + protected final static RefreshUserToGroupsMappingsRequestProto VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST = RefreshUserToGroupsMappingsRequestProto.newBuilder().build(); - private final static RefreshSuperUserGroupsConfigurationRequestProto + protected final static RefreshSuperUserGroupsConfigurationRequestProto VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST = RefreshSuperUserGroupsConfigurationRequestProto.newBuilder().build(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java index 71881ed4898c0..221b544fd402f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java @@ -33,12 +33,12 @@ public class RefreshUserMappingsProtocolServerSideTranslatorPB implements Refres private final RefreshUserMappingsProtocol impl; - private final static RefreshUserToGroupsMappingsResponseProto - VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE = + protected final static RefreshUserToGroupsMappingsResponseProto + VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE = RefreshUserToGroupsMappingsResponseProto.newBuilder().build(); - private final static RefreshSuperUserGroupsConfigurationResponseProto - VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE = + protected final static RefreshSuperUserGroupsConfigurationResponseProto + VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE = RefreshSuperUserGroupsConfigurationResponseProto.newBuilder() .build(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java index 8d8885adcba9a..43322b4563006 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java @@ -36,7 +36,7 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements ProtocolMetaInterface, GetUserMappingsProtocol, Closeable { /** RpcController is not used and hence is set to null */ - private final static RpcController NULL_CONTROLLER = null; + protected final static RpcController NULL_CONTROLLER = null; private final GetUserMappingsProtocolPB rpcProxy; public GetUserMappingsProtocolClientSideTranslatorPB( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java index f0ce85bbac873..dd9bb4c16576e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java @@ -34,7 +34,7 @@ public final class Time { /** * number of nano seconds in 1 millisecond */ - private static final long NANOSECONDS_PER_MILLISECOND = 1000000; + public static final long NANOSECONDS_PER_MILLISECOND = 1000000; private static final TimeZone UTC_ZONE = TimeZone.getTimeZone("UTC"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java index 2e0b3daa220a2..9d9d7e51d92ba 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRpcServerHandoff.java @@ -89,11 +89,11 @@ void awaitInvocation() throws InterruptedException { } void sendResponse() { - deferredCall.setDeferredResponse(request); + deferredCall.setDeferredResponse(request, 0); } void sendError() { - deferredCall.setDeferredError(new IOException("DeferredError")); + deferredCall.setDeferredError(new IOException("DeferredError"), 0); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java index 543f0a58e6ec6..28dc92341aab1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java @@ -272,49 +272,49 @@ @InterfaceStability.Stable public class ClientNamenodeProtocolTranslatorPB implements ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator { - final private ClientNamenodeProtocolPB rpcProxy; + final protected ClientNamenodeProtocolPB rpcProxy; - static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = + protected static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = GetServerDefaultsRequestProto.newBuilder().build(); - private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST = + protected final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST = GetFsStatusRequestProto.newBuilder().build(); - private final static GetFsReplicatedBlockStatsRequestProto + protected final static GetFsReplicatedBlockStatsRequestProto VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST = GetFsReplicatedBlockStatsRequestProto.newBuilder().build(); - private final static GetFsECBlockGroupStatsRequestProto + protected final static GetFsECBlockGroupStatsRequestProto VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST = GetFsECBlockGroupStatsRequestProto.newBuilder().build(); - private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = + protected final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = RollEditsRequestProto.getDefaultInstance(); - private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST = + protected final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST = RefreshNodesRequestProto.newBuilder().build(); - private final static FinalizeUpgradeRequestProto + protected final static FinalizeUpgradeRequestProto VOID_FINALIZE_UPGRADE_REQUEST = FinalizeUpgradeRequestProto.newBuilder().build(); - private final static UpgradeStatusRequestProto + protected final static UpgradeStatusRequestProto VOID_UPGRADE_STATUS_REQUEST = UpgradeStatusRequestProto.newBuilder().build(); - private final static GetDataEncryptionKeyRequestProto + protected final static GetDataEncryptionKeyRequestProto VOID_GET_DATA_ENCRYPTIONKEY_REQUEST = GetDataEncryptionKeyRequestProto.newBuilder().build(); - private final static GetStoragePoliciesRequestProto + protected final static GetStoragePoliciesRequestProto VOID_GET_STORAGE_POLICIES_REQUEST = GetStoragePoliciesRequestProto.newBuilder().build(); - private final static GetErasureCodingPoliciesRequestProto + protected final static GetErasureCodingPoliciesRequestProto VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto .newBuilder().build(); - private final static GetErasureCodingCodecsRequestProto + protected final static GetErasureCodingCodecsRequestProto VOID_GET_EC_CODEC_REQUEST = GetErasureCodingCodecsRequestProto .newBuilder().build(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java new file mode 100644 index 0000000000000..8c54f7d8b88e9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java @@ -0,0 +1,116 @@ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +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.Server; +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 static org.apache.hadoop.ipc.internal.ShadedProtobufHelper.ipc; + +public final class AsyncRpcProtocolPBUtil { + public static final Logger LOG = LoggerFactory.getLogger(AsyncRpcProtocolPBUtil.class); + public static final ThreadLocal> completableFutureThreadLocal + = new ThreadLocal<>(); + + private AsyncRpcProtocolPBUtil() {} + + public static AsyncGet asyncIpc( + ShadedProtobufHelper.IpcCall call) throws IOException { + CompletableFuture completableFuture = new CompletableFuture<>(); + Client.COMPLETABLE_FUTURE_THREAD_LOCAL.set(completableFuture); + ipc(call); + return (AsyncGet) ProtobufRpcEngine2.getAsyncReturnMessage(); + } + + public static void asyncResponse(Response response) { + CompletableFuture completableFuture = + (CompletableFuture) Client.COMPLETABLE_FUTURE_THREAD_LOCAL.get(); + // transfer originCall & callerContext to worker threads of executor. + final Server.Call originCall = Server.getCurCall().get(); + final CallerContext originContext = CallerContext.getCurrent(); + + CompletableFuture resCompletableFuture = completableFuture.thenApplyAsync(t -> { + try { + Server.getCurCall().set(originCall); + CallerContext.setCurrent(originContext); + return response.response(); + }catch (Exception e) { + throw new CompletionException(e); + } + }, RouterRpcServer.getExecutor()); + setThreadLocal(resCompletableFuture); + } + + public static void asyncRouterServer(ServerReq req, ServerRes res) { + final ProtobufRpcEngineCallback2 callback = + ProtobufRpcEngine2.Server.registerForDeferredResponse2(); + CompletableFuture completableFuture = + CompletableFuture.completedFuture(null); + // transfer originCall & callerContext to worker threads of executor. + final Server.Call originCall = Server.getCurCall().get(); + final CallerContext originContext = CallerContext.getCurrent(); + completableFuture.thenComposeAsync(o -> { + Server.getCurCall().set(originCall); + CallerContext.setCurrent(originContext); + try { + req.req(); + return (CompletableFuture)RouterAsyncRpcUtil.getCompletableFuture(); +// return CompletableFuture.completedFuture(req.req()); + } catch (Exception e) { + throw new CompletionException(e); + } + }).handle((result, e) -> { + LOG.info("zjtest async response by [{}], callback: {}, CallerContext: {}, result: [{}]", + Thread.currentThread().getName(), callback, originContext, result); + if (e == null) { + Message value = null; + try { + value = res.res(result); + } catch (RuntimeException re) { + callback.error(re); + return null; + } + callback.setResponse(value); + } else { + callback.error(e.getCause()); + } + return null; + }); + } + + public static void setThreadLocal(CompletableFuture completableFuture) { + completableFutureThreadLocal.set(completableFuture); + } + + public static CompletableFuture getCompletableFuture() { + return completableFutureThreadLocal.get(); + } + + @FunctionalInterface + interface Response { + T response() throws Exception; + } + + @FunctionalInterface + interface ServerReq { + T req() throws Exception; + } + + @FunctionalInterface + interface ServerRes { + Message res(T result) throws RuntimeException; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientNamenodeProtocolServerSideTranslatorPB.java new file mode 100644 index 0000000000000..97315d4ff2032 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientNamenodeProtocolServerSideTranslatorPB.java @@ -0,0 +1,1561 @@ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.Options; +import org.apache.hadoop.fs.permission.FsCreateModes; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.ha.proto.HAServiceProtocolProtos; +import org.apache.hadoop.hdfs.AddBlockFlag; +import org.apache.hadoop.hdfs.protocol.BatchedDirectoryListing; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; +import org.apache.hadoop.hdfs.protocol.ClientProtocol; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.HdfsPartialListing; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.OpenFilesIterator; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.io.EnumSetWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.security.proto.SecurityProtos; +import org.apache.hadoop.thirdparty.protobuf.ByteString; +import org.apache.hadoop.thirdparty.protobuf.ProtocolStringList; +import org.apache.hadoop.thirdparty.protobuf.RpcController; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncRouterServer; + +public class RouterClientNamenodeProtocolServerSideTranslatorPB + extends ClientNamenodeProtocolServerSideTranslatorPB{ + private final RouterRpcServer server; + public RouterClientNamenodeProtocolServerSideTranslatorPB( + ClientProtocol server) throws IOException { + super(server); + this.server = (RouterRpcServer) server; + } + + + @Override + public ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto getBlockLocations( + RpcController controller, ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto req) { + asyncRouterServer(() -> server.getBlockLocations(req.getSrc(), req.getOffset(), + req.getLength()), + b -> { + ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto.Builder builder + = ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto + .newBuilder(); + if (b != null) { + builder.setLocations(PBHelperClient.convert(b)).build(); + } + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto getServerDefaults( + RpcController controller, ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto req) { + asyncRouterServer(server::getServerDefaults, + result -> ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto.newBuilder() + .setServerDefaults(PBHelperClient.convert(result)) + .build()); + return null; + } + + + @Override + public ClientNamenodeProtocolProtos.CreateResponseProto create( + RpcController controller, + ClientNamenodeProtocolProtos.CreateRequestProto req) { + asyncRouterServer(() -> { + FsPermission masked = req.hasUnmasked() ? + FsCreateModes.create(PBHelperClient.convert(req.getMasked()), + PBHelperClient.convert(req.getUnmasked())) : + PBHelperClient.convert(req.getMasked()); + return server.create(req.getSrc(), + masked, req.getClientName(), + PBHelperClient.convertCreateFlag(req.getCreateFlag()), req.getCreateParent(), + (short) req.getReplication(), req.getBlockSize(), + PBHelperClient.convertCryptoProtocolVersions( + req.getCryptoProtocolVersionList()), + req.getEcPolicyName(), req.getStoragePolicy()); + }, result -> { + if (result != null) { + return ClientNamenodeProtocolProtos + .CreateResponseProto.newBuilder().setFs(PBHelperClient.convert(result)) + .build(); + } + return VOID_CREATE_RESPONSE; + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.AppendResponseProto append( + RpcController controller, + ClientNamenodeProtocolProtos.AppendRequestProto req) { + asyncRouterServer(() -> { + EnumSetWritable flags = req.hasFlag() ? + PBHelperClient.convertCreateFlag(req.getFlag()) : + new EnumSetWritable<>(EnumSet.of(CreateFlag.APPEND)); + return server.append(req.getSrc(), + req.getClientName(), flags); + }, result -> { + ClientNamenodeProtocolProtos.AppendResponseProto.Builder builder = + ClientNamenodeProtocolProtos.AppendResponseProto.newBuilder(); + if (result.getLastBlock() != null) { + builder.setBlock(PBHelperClient.convertLocatedBlock( + result.getLastBlock())); + } + if (result.getFileStatus() != null) { + builder.setStat(PBHelperClient.convert(result.getFileStatus())); + } + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SetReplicationResponseProto setReplication( + RpcController controller, + ClientNamenodeProtocolProtos.SetReplicationRequestProto req) { + asyncRouterServer(() -> + server.setReplication(req.getSrc(), (short) req.getReplication()), + result -> ClientNamenodeProtocolProtos + .SetReplicationResponseProto.newBuilder().setResult(result).build()); + return null; + } + + + @Override + public ClientNamenodeProtocolProtos.SetPermissionResponseProto setPermission( + RpcController controller, + ClientNamenodeProtocolProtos.SetPermissionRequestProto req) { + asyncRouterServer(() -> { + server.setPermission(req.getSrc(), PBHelperClient.convert(req.getPermission())); + return null; + }, result -> VOID_SET_PERM_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SetOwnerResponseProto setOwner( + RpcController controller, + ClientNamenodeProtocolProtos.SetOwnerRequestProto req) { + asyncRouterServer(() -> { + server.setOwner(req.getSrc(), + req.hasUsername() ? req.getUsername() : null, + req.hasGroupname() ? req.getGroupname() : null); + return null; + }, result -> VOID_SET_OWNER_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.AbandonBlockResponseProto abandonBlock( + RpcController controller, + ClientNamenodeProtocolProtos.AbandonBlockRequestProto req) { + asyncRouterServer(() -> { + server.abandonBlock(PBHelperClient.convert(req.getB()), req.getFileId(), + req.getSrc(), req.getHolder()); + return null; + }, result -> VOID_ADD_BLOCK_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.AddBlockResponseProto addBlock( + RpcController controller, + ClientNamenodeProtocolProtos.AddBlockRequestProto req) { + asyncRouterServer(() -> { + List excl = req.getExcludeNodesList(); + List favor = req.getFavoredNodesList(); + EnumSet flags = + PBHelperClient.convertAddBlockFlags(req.getFlagsList()); + return server.addBlock( + req.getSrc(), + req.getClientName(), + req.hasPrevious() ? PBHelperClient.convert(req.getPrevious()) : null, + (excl == null || excl.size() == 0) ? null : PBHelperClient.convert(excl + .toArray(new HdfsProtos.DatanodeInfoProto[excl.size()])), req.getFileId(), + (favor == null || favor.size() == 0) ? null : favor + .toArray(new String[favor.size()]), + flags); + }, result -> ClientNamenodeProtocolProtos.AddBlockResponseProto.newBuilder() + .setBlock(PBHelperClient.convertLocatedBlock(result)).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetAdditionalDatanodeResponseProto getAdditionalDatanode( + RpcController controller, ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto req) { + asyncRouterServer(() -> { + List existingList = req.getExistingsList(); + List existingStorageIDsList = req.getExistingStorageUuidsList(); + List excludesList = req.getExcludesList(); + LocatedBlock result = server.getAdditionalDatanode(req.getSrc(), + req.getFileId(), PBHelperClient.convert(req.getBlk()), + PBHelperClient.convert(existingList.toArray( + new HdfsProtos.DatanodeInfoProto[existingList.size()])), + existingStorageIDsList.toArray( + new String[existingStorageIDsList.size()]), + PBHelperClient.convert(excludesList.toArray( + new HdfsProtos.DatanodeInfoProto[excludesList.size()])), + req.getNumAdditionalNodes(), req.getClientName()); + return result; + }, result -> ClientNamenodeProtocolProtos + .GetAdditionalDatanodeResponseProto.newBuilder() + .setBlock( + PBHelperClient.convertLocatedBlock(result)) + .build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.CompleteResponseProto complete( + RpcController controller, + ClientNamenodeProtocolProtos.CompleteRequestProto req) { + asyncRouterServer(() -> { + boolean result = + server.complete(req.getSrc(), req.getClientName(), + req.hasLast() ? PBHelperClient.convert(req.getLast()) : null, + req.hasFileId() ? req.getFileId() : HdfsConstants.GRANDFATHER_INODE_ID); + return result; + }, result -> ClientNamenodeProtocolProtos + .CompleteResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ReportBadBlocksResponseProto reportBadBlocks( + RpcController controller, + ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto req) { + asyncRouterServer(() -> { + List bl = req.getBlocksList(); + server.reportBadBlocks(PBHelperClient.convertLocatedBlocks( + bl.toArray(new HdfsProtos.LocatedBlockProto[bl.size()]))); + return null; + }, result -> VOID_REP_BAD_BLOCK_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ConcatResponseProto concat( + RpcController controller, + ClientNamenodeProtocolProtos.ConcatRequestProto req) { + asyncRouterServer(() -> { + List srcs = req.getSrcsList(); + server.concat(req.getTrg(), srcs.toArray(new String[srcs.size()])); + return null; + }, result -> VOID_CONCAT_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RenameResponseProto rename( + RpcController controller, + ClientNamenodeProtocolProtos.RenameRequestProto req) { + asyncRouterServer(() -> { + return server.rename(req.getSrc(), req.getDst()); + }, result -> ClientNamenodeProtocolProtos + .RenameResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.Rename2ResponseProto rename2( + RpcController controller, + ClientNamenodeProtocolProtos.Rename2RequestProto req) { + asyncRouterServer(() -> { + // resolve rename options + ArrayList optionList = new ArrayList(); + if (req.getOverwriteDest()) { + optionList.add(Options.Rename.OVERWRITE); + } + if (req.hasMoveToTrash() && req.getMoveToTrash()) { + optionList.add(Options.Rename.TO_TRASH); + } + + if (optionList.isEmpty()) { + optionList.add(Options.Rename.NONE); + } + server.rename2(req.getSrc(), req.getDst(), + optionList.toArray(new Options.Rename[optionList.size()])); + return null; + }, result -> VOID_RENAME2_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.TruncateResponseProto truncate( + RpcController controller, + ClientNamenodeProtocolProtos.TruncateRequestProto req) { + asyncRouterServer(() -> server.truncate(req.getSrc(), req.getNewLength(), + req.getClientName()), result -> ClientNamenodeProtocolProtos + .TruncateResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.DeleteResponseProto delete( + RpcController controller, + ClientNamenodeProtocolProtos.DeleteRequestProto req) { + asyncRouterServer(() -> server.delete(req.getSrc(), req.getRecursive()), + result -> ClientNamenodeProtocolProtos + .DeleteResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.MkdirsResponseProto mkdirs( + RpcController controller, + ClientNamenodeProtocolProtos.MkdirsRequestProto req) { + asyncRouterServer(() -> { + FsPermission masked = req.hasUnmasked() ? + FsCreateModes.create(PBHelperClient.convert(req.getMasked()), + PBHelperClient.convert(req.getUnmasked())) : + PBHelperClient.convert(req.getMasked()); + boolean result = server.mkdirs(req.getSrc(), masked, + req.getCreateParent()); + return result; + }, result -> ClientNamenodeProtocolProtos + .MkdirsResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetListingResponseProto getListing( + RpcController controller, + ClientNamenodeProtocolProtos.GetListingRequestProto req) { + asyncRouterServer(() -> { + DirectoryListing result = server.getListing( + req.getSrc(), req.getStartAfter().toByteArray(), + req.getNeedLocation()); + return result; + }, result -> { + if (result !=null) { + return ClientNamenodeProtocolProtos + .GetListingResponseProto.newBuilder().setDirList( + PBHelperClient.convert(result)).build(); + } else { + return VOID_GETLISTING_RESPONSE; + } + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetBatchedListingResponseProto getBatchedListing( + RpcController controller, + ClientNamenodeProtocolProtos.GetBatchedListingRequestProto request) { + asyncRouterServer(() -> { + BatchedDirectoryListing result = server.getBatchedListing( + request.getPathsList().toArray(new String[]{}), + request.getStartAfter().toByteArray(), + request.getNeedLocation()); + return result; + }, result -> { + if (result != null) { + ClientNamenodeProtocolProtos.GetBatchedListingResponseProto.Builder builder = + ClientNamenodeProtocolProtos.GetBatchedListingResponseProto.newBuilder(); + for (HdfsPartialListing partialListing : result.getListings()) { + HdfsProtos.BatchedDirectoryListingProto.Builder listingBuilder = + HdfsProtos.BatchedDirectoryListingProto.newBuilder(); + if (partialListing.getException() != null) { + RemoteException ex = partialListing.getException(); + HdfsProtos.RemoteExceptionProto.Builder rexBuilder = + HdfsProtos.RemoteExceptionProto.newBuilder(); + rexBuilder.setClassName(ex.getClassName()); + if (ex.getMessage() != null) { + rexBuilder.setMessage(ex.getMessage()); + } + listingBuilder.setException(rexBuilder.build()); + } else { + for (HdfsFileStatus f : partialListing.getPartialListing()) { + listingBuilder.addPartialListing(PBHelperClient.convert(f)); + } + } + listingBuilder.setParentIdx(partialListing.getParentIdx()); + builder.addListings(listingBuilder); + } + builder.setHasMore(result.hasMore()); + builder.setStartAfter(ByteString.copyFrom(result.getStartAfter())); + return builder.build(); + } else { + return VOID_GETBATCHEDLISTING_RESPONSE; + } + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RenewLeaseResponseProto renewLease( + RpcController controller, + ClientNamenodeProtocolProtos.RenewLeaseRequestProto req) { + asyncRouterServer(() -> { + server.renewLease(req.getClientName(), req.getNamespacesList()); + return null; + }, result -> VOID_RENEWLEASE_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RecoverLeaseResponseProto recoverLease( + RpcController controller, + ClientNamenodeProtocolProtos.RecoverLeaseRequestProto req) { + asyncRouterServer(() -> server.recoverLease(req.getSrc(), req.getClientName()), + result -> ClientNamenodeProtocolProtos + .RecoverLeaseResponseProto.newBuilder() + .setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RestoreFailedStorageResponseProto restoreFailedStorage( + RpcController controller, ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto req) { + asyncRouterServer(() -> server.restoreFailedStorage(req.getArg()), + result -> ClientNamenodeProtocolProtos + .RestoreFailedStorageResponseProto.newBuilder().setResult(result) + .build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetFsStatsResponseProto getFsStats( + RpcController controller, + ClientNamenodeProtocolProtos.GetFsStatusRequestProto req) { + asyncRouterServer(server::getStats, PBHelperClient::convert); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto getFsReplicatedBlockStats( + RpcController controller, ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsRequestProto request) { + asyncRouterServer(server::getReplicatedBlockStats, PBHelperClient::convert); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsResponseProto getFsECBlockGroupStats( + RpcController controller, ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsRequestProto request) { + asyncRouterServer(server::getECBlockGroupStats, PBHelperClient::convert); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto getDatanodeReport( + RpcController controller, ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto req) { + asyncRouterServer(() -> server.getDatanodeReport(PBHelperClient.convert(req.getType())), + result -> { + List re = PBHelperClient.convert(result); + return ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto.newBuilder() + .addAllDi(re).build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto getDatanodeStorageReport( + RpcController controller, ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto req) { + asyncRouterServer(() -> server.getDatanodeStorageReport(PBHelperClient.convert(req.getType())), + result -> { + List reports = + PBHelperClient.convertDatanodeStorageReports(result); + return ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto.newBuilder() + .addAllDatanodeStorageReports(reports) + .build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto getPreferredBlockSize( + RpcController controller, ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto req) { + asyncRouterServer(() -> server.getPreferredBlockSize(req.getFilename()), + result -> ClientNamenodeProtocolProtos + .GetPreferredBlockSizeResponseProto.newBuilder().setBsize(result) + .build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SetSafeModeResponseProto setSafeMode( + RpcController controller, + ClientNamenodeProtocolProtos.SetSafeModeRequestProto req) { + asyncRouterServer(() -> server.setSafeMode(PBHelperClient.convert(req.getAction()), + req.getChecked()), + result -> ClientNamenodeProtocolProtos + .SetSafeModeResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SaveNamespaceResponseProto saveNamespace( + RpcController controller, + ClientNamenodeProtocolProtos.SaveNamespaceRequestProto req) { + asyncRouterServer(() -> { + final long timeWindow = req.hasTimeWindow() ? req.getTimeWindow() : 0; + final long txGap = req.hasTxGap() ? req.getTxGap() : 0; + return server.saveNamespace(timeWindow, txGap); + }, result -> ClientNamenodeProtocolProtos + .SaveNamespaceResponseProto.newBuilder().setSaved(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RollEditsResponseProto rollEdits( + RpcController controller, + ClientNamenodeProtocolProtos.RollEditsRequestProto request) { + asyncRouterServer(server::rollEdits, + txid -> ClientNamenodeProtocolProtos.RollEditsResponseProto.newBuilder() + .setNewSegmentTxId(txid) + .build()); + return null; + } + + + @Override + public ClientNamenodeProtocolProtos.RefreshNodesResponseProto refreshNodes( + RpcController controller, + ClientNamenodeProtocolProtos.RefreshNodesRequestProto req) { + asyncRouterServer(() -> { + server.refreshNodes(); + return null; + }, result -> VOID_REFRESHNODES_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.FinalizeUpgradeResponseProto finalizeUpgrade( + RpcController controller, + ClientNamenodeProtocolProtos.FinalizeUpgradeRequestProto req) { + asyncRouterServer(() -> { + server.finalizeUpgrade(); + return null; + }, result -> VOID_REFRESHNODES_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.UpgradeStatusResponseProto upgradeStatus( + RpcController controller, ClientNamenodeProtocolProtos.UpgradeStatusRequestProto req) { + asyncRouterServer(server::upgradeStatus, + result -> { + ClientNamenodeProtocolProtos.UpgradeStatusResponseProto.Builder b = + ClientNamenodeProtocolProtos.UpgradeStatusResponseProto.newBuilder(); + b.setUpgradeFinalized(result); + return b.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RollingUpgradeResponseProto rollingUpgrade( + RpcController controller, + ClientNamenodeProtocolProtos.RollingUpgradeRequestProto req) { + asyncRouterServer(() -> + server.rollingUpgrade(PBHelperClient.convert(req.getAction())), + info -> { + final ClientNamenodeProtocolProtos.RollingUpgradeResponseProto.Builder b = + ClientNamenodeProtocolProtos.RollingUpgradeResponseProto.newBuilder(); + if (info != null) { + b.setRollingUpgradeInfo(PBHelperClient.convert(info)); + } + return b.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto listCorruptFileBlocks( + RpcController controller, ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto req) { + asyncRouterServer(() -> server.listCorruptFileBlocks( + req.getPath(), req.hasCookie() ? req.getCookie(): null), + result -> ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto.newBuilder() + .setCorrupt(PBHelperClient.convert(result)) + .build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.MetaSaveResponseProto metaSave( + RpcController controller, + ClientNamenodeProtocolProtos.MetaSaveRequestProto req) { + asyncRouterServer(() -> { + server.metaSave(req.getFilename()); + return null; + }, result -> VOID_METASAVE_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetFileInfoResponseProto getFileInfo( + RpcController controller, + ClientNamenodeProtocolProtos.GetFileInfoRequestProto req) { + asyncRouterServer(() -> server.getFileInfo(req.getSrc()), + result -> { + if (result != null) { + return ClientNamenodeProtocolProtos.GetFileInfoResponseProto.newBuilder().setFs( + PBHelperClient.convert(result)).build(); + } + return VOID_GETFILEINFO_RESPONSE; + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetLocatedFileInfoResponseProto getLocatedFileInfo( + RpcController controller, ClientNamenodeProtocolProtos.GetLocatedFileInfoRequestProto req) { + asyncRouterServer(() -> server.getLocatedFileInfo(req.getSrc(), + req.getNeedBlockToken()), + result -> { + if (result != null) { + return ClientNamenodeProtocolProtos.GetLocatedFileInfoResponseProto.newBuilder().setFs( + PBHelperClient.convert(result)).build(); + } + return VOID_GETLOCATEDFILEINFO_RESPONSE; + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto getFileLinkInfo( + RpcController controller, + ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto req) { + asyncRouterServer(() -> server.getFileLinkInfo(req.getSrc()), + result -> { + if (result != null) { + return ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto.newBuilder().setFs( + PBHelperClient.convert(result)).build(); + } else { + return VOID_GETFILELINKINFO_RESPONSE; + } + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetContentSummaryResponseProto getContentSummary( + RpcController controller, ClientNamenodeProtocolProtos.GetContentSummaryRequestProto req) { + asyncRouterServer(() -> server.getContentSummary(req.getPath()), + result -> ClientNamenodeProtocolProtos.GetContentSummaryResponseProto.newBuilder() + .setSummary(PBHelperClient.convert(result)).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SetQuotaResponseProto setQuota( + RpcController controller, + ClientNamenodeProtocolProtos.SetQuotaRequestProto req) throws ServiceException { + asyncRouterServer(() -> { + server.setQuota(req.getPath(), req.getNamespaceQuota(), + req.getStoragespaceQuota(), + req.hasStorageType() ? + PBHelperClient.convertStorageType(req.getStorageType()): null); + return null; + }, result -> VOID_SETQUOTA_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.FsyncResponseProto fsync( + RpcController controller, + ClientNamenodeProtocolProtos.FsyncRequestProto req) { + asyncRouterServer(() -> { + server.fsync(req.getSrc(), req.getFileId(), + req.getClient(), req.getLastBlockLength()); + return null; + }, result -> VOID_FSYNC_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SetTimesResponseProto setTimes( + RpcController controller, + ClientNamenodeProtocolProtos.SetTimesRequestProto req) { + asyncRouterServer(() -> { + server.setTimes(req.getSrc(), req.getMtime(), req.getAtime()); + return null; + }, result -> VOID_SETTIMES_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.CreateSymlinkResponseProto createSymlink( + RpcController controller, + ClientNamenodeProtocolProtos.CreateSymlinkRequestProto req) { + asyncRouterServer(() -> { + server.createSymlink(req.getTarget(), req.getLink(), + PBHelperClient.convert(req.getDirPerm()), req.getCreateParent()); + return null; + }, result -> VOID_CREATESYMLINK_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetLinkTargetResponseProto getLinkTarget( + RpcController controller, + ClientNamenodeProtocolProtos.GetLinkTargetRequestProto req) { + asyncRouterServer(() -> server.getLinkTarget(req.getPath()), + result -> { + ClientNamenodeProtocolProtos.GetLinkTargetResponseProto.Builder builder = + ClientNamenodeProtocolProtos.GetLinkTargetResponseProto + .newBuilder(); + if (result != null) { + builder.setTargetPath(result); + } + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto updateBlockForPipeline( + RpcController controller, ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto req) { + asyncRouterServer(() -> server.updateBlockForPipeline(PBHelperClient.convert(req.getBlock()), + req.getClientName()), + result -> { + HdfsProtos.LocatedBlockProto res = PBHelperClient.convertLocatedBlock(result); + return ClientNamenodeProtocolProtos + .UpdateBlockForPipelineResponseProto.newBuilder().setBlock(res) + .build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.UpdatePipelineResponseProto updatePipeline( + RpcController controller, + ClientNamenodeProtocolProtos.UpdatePipelineRequestProto req) { + asyncRouterServer(() -> { + List newNodes = req.getNewNodesList(); + List newStorageIDs = req.getStorageIDsList(); + server.updatePipeline(req.getClientName(), + PBHelperClient.convert(req.getOldBlock()), + PBHelperClient.convert(req.getNewBlock()), + PBHelperClient.convert(newNodes.toArray(new HdfsProtos.DatanodeIDProto[newNodes.size()])), + newStorageIDs.toArray(new String[newStorageIDs.size()])); + return null; + }, result -> VOID_UPDATEPIPELINE_RESPONSE); + return null; + } + + @Override + public SecurityProtos.GetDelegationTokenResponseProto getDelegationToken( + RpcController controller, SecurityProtos.GetDelegationTokenRequestProto req) { + asyncRouterServer(() -> server + .getDelegationToken(new Text(req.getRenewer())), + token -> { + SecurityProtos.GetDelegationTokenResponseProto.Builder rspBuilder = + SecurityProtos.GetDelegationTokenResponseProto.newBuilder(); + if (token != null) { + rspBuilder.setToken(PBHelperClient.convert(token)); + } + return rspBuilder.build(); + }); + return null; + } + + @Override + public SecurityProtos.RenewDelegationTokenResponseProto renewDelegationToken( + RpcController controller, SecurityProtos.RenewDelegationTokenRequestProto req) { + asyncRouterServer(() -> server.renewDelegationToken(PBHelperClient + .convertDelegationToken(req.getToken())), + result -> SecurityProtos.RenewDelegationTokenResponseProto.newBuilder() + .setNewExpiryTime(result).build()); + return null; + } + + @Override + public SecurityProtos.CancelDelegationTokenResponseProto cancelDelegationToken( + RpcController controller, SecurityProtos.CancelDelegationTokenRequestProto req) { + asyncRouterServer(() -> { + server.cancelDelegationToken(PBHelperClient.convertDelegationToken(req + .getToken())); + return null; + }, result -> VOID_CANCELDELEGATIONTOKEN_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SetBalancerBandwidthResponseProto setBalancerBandwidth( + RpcController controller, ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto req) { + asyncRouterServer(() -> { + server.setBalancerBandwidth(req.getBandwidth()); + return null; + }, result -> VOID_SETBALANCERBANDWIDTH_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto getDataEncryptionKey( + RpcController controller, ClientNamenodeProtocolProtos.GetDataEncryptionKeyRequestProto request) { + asyncRouterServer(server::getDataEncryptionKey, encryptionKey -> { + ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto.Builder builder = + ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto.newBuilder(); + if (encryptionKey != null) { + builder.setDataEncryptionKey(PBHelperClient.convert(encryptionKey)); + } + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.CreateSnapshotResponseProto createSnapshot( + RpcController controller, + ClientNamenodeProtocolProtos.CreateSnapshotRequestProto req) throws ServiceException { + asyncRouterServer(() -> server.createSnapshot(req.getSnapshotRoot(), + req.hasSnapshotName()? req.getSnapshotName(): null), + snapshotPath -> { + final ClientNamenodeProtocolProtos.CreateSnapshotResponseProto.Builder builder + = ClientNamenodeProtocolProtos.CreateSnapshotResponseProto.newBuilder(); + if (snapshotPath != null) { + builder.setSnapshotPath(snapshotPath); + } + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.DeleteSnapshotResponseProto deleteSnapshot( + RpcController controller, + ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto req) { + asyncRouterServer(() -> { + server.deleteSnapshot(req.getSnapshotRoot(), req.getSnapshotName()); + return null; + }, result -> VOID_DELETE_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.AllowSnapshotResponseProto allowSnapshot( + RpcController controller, + ClientNamenodeProtocolProtos.AllowSnapshotRequestProto req) { + asyncRouterServer(() -> { + server.allowSnapshot(req.getSnapshotRoot()); + return null; + }, result -> VOID_ALLOW_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.DisallowSnapshotResponseProto disallowSnapshot( + RpcController controller, + ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto req) { + asyncRouterServer(() -> { + server.disallowSnapshot(req.getSnapshotRoot()); + return null; + }, result -> VOID_DISALLOW_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RenameSnapshotResponseProto renameSnapshot( + RpcController controller, + ClientNamenodeProtocolProtos.RenameSnapshotRequestProto request) { + asyncRouterServer(() -> { + server.renameSnapshot(request.getSnapshotRoot(), + request.getSnapshotOldName(), request.getSnapshotNewName()); + return null; + }, result -> VOID_RENAME_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto getSnapshottableDirListing( + RpcController controller, ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto request) { + asyncRouterServer(server::getSnapshottableDirListing, + result -> { + if (result != null) { + return ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto.newBuilder(). + setSnapshottableDirList(PBHelperClient.convert(result)).build(); + } else { + return NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE; + } + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetSnapshotListingResponseProto getSnapshotListing( + RpcController controller, ClientNamenodeProtocolProtos.GetSnapshotListingRequestProto request) { + asyncRouterServer(() -> server + .getSnapshotListing(request.getSnapshotRoot()), + result -> { + if (result != null) { + return ClientNamenodeProtocolProtos.GetSnapshotListingResponseProto.newBuilder(). + setSnapshotList(PBHelperClient.convert(result)).build(); + } else { + return NULL_GET_SNAPSHOT_LISTING_RESPONSE; + } + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto getSnapshotDiffReport( + RpcController controller, ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto request) { + asyncRouterServer(() -> server.getSnapshotDiffReport( + request.getSnapshotRoot(), request.getFromSnapshot(), + request.getToSnapshot()), + report -> ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto.newBuilder() + .setDiffReport(PBHelperClient.convert(report)).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto getSnapshotDiffReportListing( + RpcController controller, + ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto request) { + asyncRouterServer(() -> server + .getSnapshotDiffReportListing(request.getSnapshotRoot(), + request.getFromSnapshot(), request.getToSnapshot(), + request.getCursor().getStartPath().toByteArray(), + request.getCursor().getIndex()), + report -> ClientNamenodeProtocolProtos + .GetSnapshotDiffReportListingResponseProto.newBuilder() + .setDiffReport(PBHelperClient.convert(report)).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.IsFileClosedResponseProto isFileClosed( + RpcController controller, ClientNamenodeProtocolProtos.IsFileClosedRequestProto request) { + asyncRouterServer(() -> server.isFileClosed(request.getSrc()), + result -> ClientNamenodeProtocolProtos + .IsFileClosedResponseProto.newBuilder() + .setResult(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto addCacheDirective( + RpcController controller, ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto request) { + asyncRouterServer(() -> server.addCacheDirective( + PBHelperClient.convert(request.getInfo()), + PBHelperClient.convertCacheFlags(request.getCacheFlags())), + id -> ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto.newBuilder(). + setId(id).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto modifyCacheDirective( + RpcController controller, ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto request) { + asyncRouterServer(() -> { + server.modifyCacheDirective( + PBHelperClient.convert(request.getInfo()), + PBHelperClient.convertCacheFlags(request.getCacheFlags())); + return null; + }, result -> ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto.newBuilder().build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto + removeCacheDirective( + RpcController controller, + ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto request) { + asyncRouterServer(() -> { + server.removeCacheDirective(request.getId()); + return null; + }, result -> ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto. + newBuilder().build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto listCacheDirectives( + RpcController controller, ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto request) { + asyncRouterServer(() -> { + CacheDirectiveInfo filter = + PBHelperClient.convert(request.getFilter()); + return server.listCacheDirectives(request.getPrevId(), filter); + }, entries -> { + ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto.Builder builder = + ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto.newBuilder(); + builder.setHasMore(entries.hasMore()); + for (int i=0, n=entries.size(); i { + server.addCachePool(PBHelperClient.convert(request.getInfo())); + return null; + }, result -> ClientNamenodeProtocolProtos.AddCachePoolResponseProto.newBuilder().build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto modifyCachePool( + RpcController controller, + ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto request) { + asyncRouterServer(() -> { + server.modifyCachePool(PBHelperClient.convert(request.getInfo())); + return null; + }, result -> ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto.newBuilder().build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto removeCachePool( + RpcController controller, + ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto request) { + asyncRouterServer(() -> { + server.removeCachePool(request.getPoolName()); + return null; + }, result -> ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto.newBuilder().build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ListCachePoolsResponseProto listCachePools( + RpcController controller, + ClientNamenodeProtocolProtos.ListCachePoolsRequestProto request) { + asyncRouterServer(() -> server.listCachePools(request.getPrevPoolName()), + entries -> { + ClientNamenodeProtocolProtos.ListCachePoolsResponseProto.Builder responseBuilder = + ClientNamenodeProtocolProtos.ListCachePoolsResponseProto.newBuilder(); + responseBuilder.setHasMore(entries.hasMore()); + for (int i=0, n=entries.size(); i { + server.modifyAclEntries(req.getSrc(), PBHelperClient.convertAclEntry(req.getAclSpecList())); + return null; + }, vo -> VOID_MODIFYACLENTRIES_RESPONSE); + return null; + } + + @Override + public AclProtos.RemoveAclEntriesResponseProto removeAclEntries( + RpcController controller, AclProtos.RemoveAclEntriesRequestProto req) { + asyncRouterServer(() -> { + server.removeAclEntries(req.getSrc(), + PBHelperClient.convertAclEntry(req.getAclSpecList())); + return null; + }, vo -> VOID_REMOVEACLENTRIES_RESPONSE); + return null; + } + + @Override + public AclProtos.RemoveDefaultAclResponseProto removeDefaultAcl( + RpcController controller, AclProtos.RemoveDefaultAclRequestProto req) { + asyncRouterServer(() -> { + server.removeDefaultAcl(req.getSrc()); + return null; + }, vo -> VOID_REMOVEDEFAULTACL_RESPONSE); + return null; + } + + @Override + public AclProtos.RemoveAclResponseProto removeAcl( + RpcController controller, + AclProtos.RemoveAclRequestProto req) { + asyncRouterServer(() -> { + server.removeAcl(req.getSrc()); + return null; + }, vo -> VOID_REMOVEACL_RESPONSE); + return null; + } + + @Override + public AclProtos.SetAclResponseProto setAcl( + RpcController controller, + AclProtos.SetAclRequestProto req) { + asyncRouterServer(() -> { + server.setAcl(req.getSrc(), PBHelperClient.convertAclEntry(req.getAclSpecList())); + return null; + }, vo -> VOID_SETACL_RESPONSE); + return null; + } + + @Override + public AclProtos.GetAclStatusResponseProto getAclStatus( + RpcController controller, + AclProtos.GetAclStatusRequestProto req) { + asyncRouterServer(() -> server.getAclStatus(req.getSrc()), + PBHelperClient::convert); + return null; + } + + @Override + public EncryptionZonesProtos.CreateEncryptionZoneResponseProto createEncryptionZone( + RpcController controller, EncryptionZonesProtos.CreateEncryptionZoneRequestProto req) { + asyncRouterServer(() -> { + server.createEncryptionZone(req.getSrc(), req.getKeyName()); + return null; + }, vo -> EncryptionZonesProtos.CreateEncryptionZoneResponseProto.newBuilder().build()); + return null; + } + + @Override + public EncryptionZonesProtos.GetEZForPathResponseProto getEZForPath( + RpcController controller, EncryptionZonesProtos.GetEZForPathRequestProto req) { + asyncRouterServer(() -> server.getEZForPath(req.getSrc()), + ret -> { + EncryptionZonesProtos.GetEZForPathResponseProto.Builder builder = + EncryptionZonesProtos.GetEZForPathResponseProto.newBuilder(); + if (ret != null) { + builder.setZone(PBHelperClient.convert(ret)); + } + return builder.build(); + }); + return null; + } + + @Override + public EncryptionZonesProtos.ListEncryptionZonesResponseProto listEncryptionZones( + RpcController controller, EncryptionZonesProtos.ListEncryptionZonesRequestProto req) { + asyncRouterServer(() -> server.listEncryptionZones(req.getId()), + entries -> { + EncryptionZonesProtos.ListEncryptionZonesResponseProto.Builder builder = + EncryptionZonesProtos.ListEncryptionZonesResponseProto.newBuilder(); + builder.setHasMore(entries.hasMore()); + for (int i=0; i { + server.reencryptEncryptionZone(req.getZone(), + PBHelperClient.convert(req.getAction())); + return null; + }, vo -> EncryptionZonesProtos.ReencryptEncryptionZoneResponseProto.newBuilder().build()); + return null; + } + + public EncryptionZonesProtos.ListReencryptionStatusResponseProto listReencryptionStatus( + RpcController controller, EncryptionZonesProtos.ListReencryptionStatusRequestProto req) { + asyncRouterServer(() -> server.listReencryptionStatus(req.getId()), + entries -> { + EncryptionZonesProtos.ListReencryptionStatusResponseProto.Builder builder = + EncryptionZonesProtos.ListReencryptionStatusResponseProto.newBuilder(); + builder.setHasMore(entries.hasMore()); + for (int i=0; i { + String ecPolicyName = req.hasEcPolicyName() ? + req.getEcPolicyName() : null; + server.setErasureCodingPolicy(req.getSrc(), ecPolicyName); + return null; + }, vo -> ErasureCodingProtos + .SetErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public ErasureCodingProtos.UnsetErasureCodingPolicyResponseProto unsetErasureCodingPolicy( + RpcController controller, ErasureCodingProtos.UnsetErasureCodingPolicyRequestProto req) { + asyncRouterServer(() -> { + server.unsetErasureCodingPolicy(req.getSrc()); + return null; + }, vo -> ErasureCodingProtos + .UnsetErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public ErasureCodingProtos.GetECTopologyResultForPoliciesResponseProto getECTopologyResultForPolicies( + RpcController controller, ErasureCodingProtos.GetECTopologyResultForPoliciesRequestProto req) { + asyncRouterServer(() -> { + ProtocolStringList policies = req.getPoliciesList(); + return server.getECTopologyResultForPolicies( + policies.toArray(policies.toArray(new String[policies.size()]))); + }, result -> { + ErasureCodingProtos.GetECTopologyResultForPoliciesResponseProto.Builder builder = + ErasureCodingProtos.GetECTopologyResultForPoliciesResponseProto.newBuilder(); + builder + .setResponse(PBHelperClient.convertECTopologyVerifierResult(result)); + return builder.build(); + }); + return null; + } + + @Override + public XAttrProtos.SetXAttrResponseProto setXAttr( + RpcController controller, + XAttrProtos.SetXAttrRequestProto req) { + asyncRouterServer(() -> { + server.setXAttr(req.getSrc(), PBHelperClient.convertXAttr(req.getXAttr()), + PBHelperClient.convert(req.getFlag())); + return null; + }, vo -> VOID_SETXATTR_RESPONSE); + return null; + } + + @Override + public XAttrProtos.GetXAttrsResponseProto getXAttrs( + RpcController controller, + XAttrProtos.GetXAttrsRequestProto req) { + asyncRouterServer(() -> server.getXAttrs(req.getSrc(), + PBHelperClient.convertXAttrs(req.getXAttrsList())), + PBHelperClient::convertXAttrsResponse); + return null; + } + + @Override + public XAttrProtos.ListXAttrsResponseProto listXAttrs( + RpcController controller, + XAttrProtos.ListXAttrsRequestProto req) { + asyncRouterServer(() -> server.listXAttrs(req.getSrc()), + PBHelperClient::convertListXAttrsResponse); + return null; + } + + @Override + public XAttrProtos.RemoveXAttrResponseProto removeXAttr( + RpcController controller, + XAttrProtos.RemoveXAttrRequestProto req) { + asyncRouterServer(() -> { + server.removeXAttr(req.getSrc(), PBHelperClient.convertXAttr(req.getXAttr())); + return null; + }, vo -> VOID_REMOVEXATTR_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.CheckAccessResponseProto checkAccess( + RpcController controller, + ClientNamenodeProtocolProtos.CheckAccessRequestProto req) { + asyncRouterServer(() -> { + server.checkAccess(req.getPath(), PBHelperClient.convert(req.getMode())); + return null; + }, vo -> VOID_CHECKACCESS_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SetStoragePolicyResponseProto setStoragePolicy( + RpcController controller, ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto request) { + asyncRouterServer(() -> { + server.setStoragePolicy(request.getSrc(), request.getPolicyName()); + return null; + }, vo -> VOID_SET_STORAGE_POLICY_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.UnsetStoragePolicyResponseProto unsetStoragePolicy( + RpcController controller, ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto request) { + asyncRouterServer(() -> { + server.unsetStoragePolicy(request.getSrc()); + return null; + }, vo -> VOID_UNSET_STORAGE_POLICY_RESPONSE); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetStoragePolicyResponseProto getStoragePolicy( + RpcController controller, ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto request) { + asyncRouterServer(() -> server.getStoragePolicy(request.getPath()), + result -> { + HdfsProtos.BlockStoragePolicyProto policy = PBHelperClient.convert(result); + return ClientNamenodeProtocolProtos.GetStoragePolicyResponseProto.newBuilder() + .setStoragePolicy(policy).build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto getStoragePolicies( + RpcController controller, ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto request) { + asyncRouterServer(server::getStoragePolicies, + policies -> { + ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto.Builder builder = + ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto.newBuilder(); + if (policies == null) { + return builder.build(); + } + for (BlockStoragePolicy policy : policies) { + builder.addPolicies(PBHelperClient.convert(policy)); + } + return builder.build(); + }); + return null; + } + + public ClientNamenodeProtocolProtos.GetCurrentEditLogTxidResponseProto getCurrentEditLogTxid( + RpcController controller, + ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto req) throws ServiceException { + asyncRouterServer(server::getCurrentEditLogTxid, + result -> ClientNamenodeProtocolProtos + .GetCurrentEditLogTxidResponseProto.newBuilder() + .setTxid(result).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto getEditsFromTxid( + RpcController controller, + ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto req) { + asyncRouterServer(() -> server.getEditsFromTxid(req.getTxid()), + PBHelperClient::convertEditsResponse); + return null; + } + + @Override + public ErasureCodingProtos.GetErasureCodingPoliciesResponseProto getErasureCodingPolicies( + RpcController controller, + ErasureCodingProtos.GetErasureCodingPoliciesRequestProto request) { + asyncRouterServer(server::getErasureCodingPolicies, + ecpInfos -> { + ErasureCodingProtos.GetErasureCodingPoliciesResponseProto.Builder resBuilder = + ErasureCodingProtos.GetErasureCodingPoliciesResponseProto + .newBuilder(); + for (ErasureCodingPolicyInfo info : ecpInfos) { + resBuilder.addEcPolicies( + PBHelperClient.convertErasureCodingPolicy(info)); + } + return resBuilder.build(); + }); + return null; + } + + @Override + public ErasureCodingProtos.GetErasureCodingCodecsResponseProto getErasureCodingCodecs( + RpcController controller, ErasureCodingProtos.GetErasureCodingCodecsRequestProto request) { + asyncRouterServer(server::getErasureCodingCodecs, + codecs -> { + ErasureCodingProtos.GetErasureCodingCodecsResponseProto.Builder resBuilder = + ErasureCodingProtos.GetErasureCodingCodecsResponseProto.newBuilder(); + for (Map.Entry codec : codecs.entrySet()) { + resBuilder.addCodec( + PBHelperClient.convertErasureCodingCodec( + codec.getKey(), codec.getValue())); + } + return resBuilder.build(); + }); + return null; + } + + @Override + public ErasureCodingProtos.AddErasureCodingPoliciesResponseProto addErasureCodingPolicies( + RpcController controller, ErasureCodingProtos.AddErasureCodingPoliciesRequestProto request) { + asyncRouterServer(() -> { + ErasureCodingPolicy[] policies = request.getEcPoliciesList().stream() + .map(PBHelperClient::convertErasureCodingPolicy) + .toArray(ErasureCodingPolicy[]::new); + return server + .addErasureCodingPolicies(policies); + }, result -> { + List responseProtos = + Arrays.stream(result) + .map(PBHelperClient::convertAddErasureCodingPolicyResponse) + .collect(Collectors.toList()); + ErasureCodingProtos.AddErasureCodingPoliciesResponseProto response = + ErasureCodingProtos.AddErasureCodingPoliciesResponseProto.newBuilder() + .addAllResponses(responseProtos).build(); + return response; + }); + return null; + } + + @Override + public ErasureCodingProtos.RemoveErasureCodingPolicyResponseProto removeErasureCodingPolicy( + RpcController controller, ErasureCodingProtos.RemoveErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> { + server.removeErasureCodingPolicy(request.getEcPolicyName()); + return null; + }, vo -> ErasureCodingProtos.RemoveErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public ErasureCodingProtos.EnableErasureCodingPolicyResponseProto enableErasureCodingPolicy( + RpcController controller, ErasureCodingProtos.EnableErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> { + server.enableErasureCodingPolicy(request.getEcPolicyName()); + return null; + }, vo -> ErasureCodingProtos.EnableErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public ErasureCodingProtos.DisableErasureCodingPolicyResponseProto disableErasureCodingPolicy( + RpcController controller, ErasureCodingProtos.DisableErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> { + server.disableErasureCodingPolicy(request.getEcPolicyName()); + return null; + }, vo -> ErasureCodingProtos.DisableErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public ErasureCodingProtos.GetErasureCodingPolicyResponseProto getErasureCodingPolicy( + RpcController controller, + ErasureCodingProtos.GetErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> server.getErasureCodingPolicy(request.getSrc()), + ecPolicy -> { + ErasureCodingProtos.GetErasureCodingPolicyResponseProto.Builder builder = + ErasureCodingProtos.GetErasureCodingPolicyResponseProto.newBuilder(); + if (ecPolicy != null) { + builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy)); + } + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetQuotaUsageResponseProto getQuotaUsage( + RpcController controller, ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto req) { + asyncRouterServer(() -> server.getQuotaUsage(req.getPath()), + result -> ClientNamenodeProtocolProtos.GetQuotaUsageResponseProto.newBuilder() + .setUsage(PBHelperClient.convert(result)).build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.ListOpenFilesResponseProto listOpenFiles( + RpcController controller, + ClientNamenodeProtocolProtos.ListOpenFilesRequestProto req) { + asyncRouterServer(() -> { + EnumSet openFilesTypes = + PBHelperClient.convertOpenFileTypes(req.getTypesList()); + return server.listOpenFiles(req.getId(), + openFilesTypes, req.getPath()); + }, entries -> { + ClientNamenodeProtocolProtos.ListOpenFilesResponseProto.Builder builder = + ClientNamenodeProtocolProtos.ListOpenFilesResponseProto.newBuilder(); + builder.setHasMore(entries.hasMore()); + for (int i = 0; i < entries.size(); i++) { + builder.addEntries(PBHelperClient.convert(entries.get(i))); + } + builder.addAllTypes(req.getTypesList()); + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.MsyncResponseProto msync( + RpcController controller, + ClientNamenodeProtocolProtos.MsyncRequestProto req) { + asyncRouterServer(() -> { + server.msync(); + return null; + }, vo -> ClientNamenodeProtocolProtos.MsyncResponseProto.newBuilder().build()); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.SatisfyStoragePolicyResponseProto satisfyStoragePolicy( + RpcController controller, + ClientNamenodeProtocolProtos.SatisfyStoragePolicyRequestProto request) throws ServiceException { + try { + server.satisfyStoragePolicy(request.getSrc()); + } catch (IOException e) { + throw new ServiceException(e); + } + return VOID_SATISFYSTORAGEPOLICY_RESPONSE; + } + + @Override + public ClientNamenodeProtocolProtos.HAServiceStateResponseProto getHAServiceState( + RpcController controller, + ClientNamenodeProtocolProtos.HAServiceStateRequestProto request) { + asyncRouterServer(server::getHAServiceState, + state -> { + HAServiceProtocolProtos.HAServiceStateProto retState; + switch (state) { + case ACTIVE: + retState = HAServiceProtocolProtos.HAServiceStateProto.ACTIVE; + break; + case STANDBY: + retState = HAServiceProtocolProtos.HAServiceStateProto.STANDBY; + break; + case OBSERVER: + retState = HAServiceProtocolProtos.HAServiceStateProto.OBSERVER; + break; + case INITIALIZING: + default: + retState = HAServiceProtocolProtos.HAServiceStateProto.INITIALIZING; + break; + } + ClientNamenodeProtocolProtos.HAServiceStateResponseProto.Builder builder = + ClientNamenodeProtocolProtos.HAServiceStateResponseProto.newBuilder(); + builder.setState(retState); + return builder.build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetSlowDatanodeReportResponseProto getSlowDatanodeReport( + RpcController controller, + ClientNamenodeProtocolProtos.GetSlowDatanodeReportRequestProto request) { + asyncRouterServer(server::getSlowDatanodeReport, + res -> { + List result = PBHelperClient.convert(res); + return ClientNamenodeProtocolProtos.GetSlowDatanodeReportResponseProto.newBuilder() + .addAllDatanodeInfoProto(result) + .build(); + }); + return null; + } + + @Override + public ClientNamenodeProtocolProtos.GetEnclosingRootResponseProto getEnclosingRoot( + RpcController controller, ClientNamenodeProtocolProtos.GetEnclosingRootRequestProto req) { + asyncRouterServer(() -> server.getEnclosingRoot(req.getFilename()), + enclosingRootPath -> ClientNamenodeProtocolProtos + .GetEnclosingRootResponseProto.newBuilder() + .setEnclosingRootPath(enclosingRootPath.toUri().toString()) + .build()); + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientProtocolTranslatorPB.java new file mode 100644 index 0000000000000..a1683d1578f60 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientProtocolTranslatorPB.java @@ -0,0 +1,2705 @@ +/** + * 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 java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; + +import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.crypto.CryptoProtocolVersion; +import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries; +import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries; +import org.apache.hadoop.fs.CacheFlag; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FsServerDefaults; +import org.apache.hadoop.fs.Options.Rename; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.fs.XAttr; +import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.AclStatus; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.ha.HAServiceProtocol; +import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto; +import org.apache.hadoop.hdfs.AddBlockFlag; +import org.apache.hadoop.hdfs.inotify.EventBatchList; +import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; +import org.apache.hadoop.hdfs.protocol.BatchedDirectoryListing; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; +import org.apache.hadoop.hdfs.protocol.CachePoolEntry; +import org.apache.hadoop.hdfs.protocol.CachePoolInfo; +import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; +import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.HdfsPartialListing; +import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats; +import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult; +import org.apache.hadoop.hdfs.protocol.EncryptionZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; +import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType; +import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats; +import org.apache.hadoop.hdfs.protocol.OpenFileEntry; +import org.apache.hadoop.hdfs.protocol.OpenFilesIterator; +import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus; +import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing; +import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; +import org.apache.hadoop.hdfs.protocol.SnapshotStatus; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBatchedListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBatchedListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSlowDatanodeReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSlowDatanodeReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListOpenFilesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListOpenFilesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.OpenFilesBatchResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListReencryptionStatusRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListReencryptionStatusResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ZoneReencryptionStatusProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptEncryptionZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.RemoveErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.EnableErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.DisableErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingCodecsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CodecProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BatchedDirectoryListingProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECTopologyResultForPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECTopologyResultForPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto; +import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; +import org.apache.hadoop.io.EnumSetWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.ipc.RpcClientUtil; +import org.apache.hadoop.security.proto.SecurityProtos; +import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto; +import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; +import org.apache.hadoop.security.token.Token; + +import org.apache.hadoop.thirdparty.protobuf.ByteString; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; + +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.concurrent.AsyncGet; +import org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.Response; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; +import static org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getRemoteException; +import static org.apache.hadoop.ipc.internal.ShadedProtobufHelper.ipc; + + +/** + * This class forwards NN's ClientProtocol calls as RPC calls to the NN server + * while translating from the parameter types used in ClientProtocol to the + * new PB types. + */ +@InterfaceAudience.Private +@InterfaceStability.Stable +public class RouterClientProtocolTranslatorPB extends ClientNamenodeProtocolTranslatorPB { + + public RouterClientProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) { + super(proxy); + } + + @Override + public void close() { + super.close(); + } + + @Override + public LocatedBlocks getBlockLocations(String src, long offset, long length) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBlockLocations(src, offset, length); + } + GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto + .newBuilder() + .setSrc(src) + .setOffset(offset) + .setLength(length) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getBlockLocations(null, req)); + asyncResponse(() -> { + GetBlockLocationsResponseProto resp = asyncGet.get(-1, null); + return resp.hasLocations() ? + PBHelperClient.convert(resp.getLocations()) : null; + }); + return null; + } + + @Override + public FsServerDefaults getServerDefaults() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getServerDefaults(); + } + GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST; + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getServerDefaults(null, req)); + asyncResponse(() -> + PBHelperClient.convert(asyncGet.get(-1, null).getServerDefaults())); + return null; + } + + @Override + public HdfsFileStatus create( + String src, FsPermission masked, + String clientName, EnumSetWritable flag, + boolean createParent, short replication, long blockSize, + CryptoProtocolVersion[] supportedVersions, String ecPolicyName, + String storagePolicy) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.create( + src, masked, clientName, flag, createParent, replication, + blockSize, supportedVersions, ecPolicyName, storagePolicy); + } + + CreateRequestProto.Builder builder = CreateRequestProto.newBuilder() + .setSrc(src) + .setMasked(PBHelperClient.convert(masked)) + .setClientName(clientName) + .setCreateFlag(PBHelperClient.convertCreateFlag(flag)) + .setCreateParent(createParent) + .setReplication(replication) + .setBlockSize(blockSize); + if (ecPolicyName != null) { + builder.setEcPolicyName(ecPolicyName); + } + if (storagePolicy != null) { + builder.setStoragePolicy(storagePolicy); + } + FsPermission unmasked = masked.getUnmasked(); + if (unmasked != null) { + builder.setUnmasked(PBHelperClient.convert(unmasked)); + } + builder.addAllCryptoProtocolVersion( + PBHelperClient.convert(supportedVersions)); + CreateRequestProto req = builder.build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.create(null, req)); + asyncResponse(() -> { + CreateResponseProto res = asyncGet.get(-1, null); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + }); + return null; + } + + @Override + public boolean truncate(String src, long newLength, String clientName) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.truncate(src, newLength, clientName); + } + + TruncateRequestProto req = TruncateRequestProto.newBuilder() + .setSrc(src) + .setNewLength(newLength) + .setClientName(clientName) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.truncate(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + + @Override + public LastBlockWithStatus append(String src, String clientName, + EnumSetWritable flag) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.append(src, clientName, flag); + } + + AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src) + .setClientName(clientName).setFlag( + PBHelperClient.convertCreateFlag(flag)) + .build(); + + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.append(null, req)); + asyncResponse(() -> { + AppendResponseProto res = asyncGet.get(-1, null); + LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient + .convertLocatedBlockProto(res.getBlock()) : null; + HdfsFileStatus stat = (res.hasStat()) ? + PBHelperClient.convert(res.getStat()) : null; + return new LastBlockWithStatus(lastBlock, stat); + }); + return null; + } + + @Override + public boolean setReplication(String src, short replication) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.setReplication(src, replication); + } + + SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder() + .setSrc(src) + .setReplication(replication) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setReplication(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + + @Override + public void setPermission(String src, FsPermission permission) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setPermission(src, permission); + return; + } + + SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder() + .setSrc(src) + .setPermission(PBHelperClient.convert(permission)) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setPermission(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public void setOwner(String src, String username, String groupname) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setOwner(src, username, groupname); + } + + SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder() + .setSrc(src); + if (username != null) { + req.setUsername(username); + } + if (groupname != null) { + req.setGroupname(groupname); + } + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setOwner(null, req.build())); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public void abandonBlock(ExtendedBlock b, long fileId, String src, + String holder) throws IOException { + if (!Client.isAsynchronousMode()) { + super.abandonBlock(b, fileId, src, holder); + } + AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder() + .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder) + .setFileId(fileId).build(); + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.abandonBlock(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public LocatedBlock addBlock( + String src, String clientName, + ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, + String[] favoredNodes, EnumSet addBlockFlags) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.addBlock(src, clientName, previous, excludeNodes, + fileId, favoredNodes, addBlockFlags); + } + AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder() + .setSrc(src).setClientName(clientName).setFileId(fileId); + if (previous != null) { + req.setPrevious(PBHelperClient.convert(previous)); + } + if (excludeNodes != null) { + req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes)); + } + if (favoredNodes != null) { + req.addAllFavoredNodes(Arrays.asList(favoredNodes)); + } + if (addBlockFlags != null) { + req.addAllFlags(PBHelperClient.convertAddBlockFlags( + addBlockFlags)); + } + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.addBlock(null, req.build())); + asyncResponse(() -> PBHelperClient.convertLocatedBlockProto( + asyncGet.get(-1, null).getBlock())); + return null; + } + + @Override + public LocatedBlock getAdditionalDatanode( + String src, long fileId, + ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs, + DatanodeInfo[] excludes, int numAdditionalNodes, String clientName) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getAdditionalDatanode(src, fileId, blk, existings, + existingStorageIDs, excludes, numAdditionalNodes, clientName); + } + GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto + .newBuilder() + .setSrc(src) + .setFileId(fileId) + .setBlk(PBHelperClient.convert(blk)) + .addAllExistings(PBHelperClient.convert(existings)) + .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs)) + .addAllExcludes(PBHelperClient.convert(excludes)) + .setNumAdditionalNodes(numAdditionalNodes) + .setClientName(clientName) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getAdditionalDatanode(null, req)); + asyncResponse(() -> PBHelperClient.convertLocatedBlockProto( + asyncGet.get(-1, null).getBlock())); + return null; + } + + @Override + public boolean complete(String src, String clientName, + ExtendedBlock last, long fileId) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.complete(src, clientName, last, fileId); + } + CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder() + .setSrc(src) + .setClientName(clientName) + .setFileId(fileId); + if (last != null) { + req.setLast(PBHelperClient.convert(last)); + } + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.complete(null, req.build())); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + + @Override + public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { + if (!Client.isAsynchronousMode()) { + super.reportBadBlocks(blocks); + return; + } + ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder() + .addAllBlocks(Arrays.asList( + PBHelperClient.convertLocatedBlocks(blocks))) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.reportBadBlocks(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public boolean rename(String src, String dst) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.rename(src, dst); + } + RenameRequestProto req = RenameRequestProto.newBuilder() + .setSrc(src) + .setDst(dst).build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rename(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getResult()); + return true; + } + + + @Override + public void rename2(String src, String dst, Rename... options) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.rename2(src, dst, options); + return; + } + boolean overwrite = false; + boolean toTrash = false; + if (options != null) { + for (Rename option : options) { + if (option == Rename.OVERWRITE) { + overwrite = true; + } + if (option == Rename.TO_TRASH) { + toTrash = true; + } + } + } + Rename2RequestProto req = Rename2RequestProto.newBuilder(). + setSrc(src). + setDst(dst). + setOverwriteDest(overwrite). + setMoveToTrash(toTrash). + build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rename2(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public void concat(String trg, String[] srcs) throws IOException { + ConcatRequestProto req = ConcatRequestProto.newBuilder(). + setTrg(trg). + addAllSrcs(Arrays.asList(srcs)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.concat(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.concat(null, req)); + } + } + + + @Override + public boolean delete(String src, boolean recursive) throws IOException { + DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src) + .setRecursive(recursive).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.delete(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.delete(null, req).getResult()); + } + + @Override + public boolean mkdirs(String src, FsPermission masked, boolean createParent) + throws IOException { + MkdirsRequestProto.Builder builder = MkdirsRequestProto.newBuilder() + .setSrc(src) + .setMasked(PBHelperClient.convert(masked)) + .setCreateParent(createParent); + FsPermission unmasked = masked.getUnmasked(); + if (unmasked != null) { + builder.setUnmasked(PBHelperClient.convert(unmasked)); + } + MkdirsRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.mkdirs(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.mkdirs(null, req)).getResult(); + } + + @Override + public DirectoryListing getListing(String src, byte[] startAfter, + boolean needLocation) throws IOException { + GetListingRequestProto req = GetListingRequestProto.newBuilder() + .setSrc(src) + .setStartAfter(ByteString.copyFrom(startAfter)) + .setNeedLocation(needLocation).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getListing(null, req)); + asyncResponse(() -> { + GetListingResponseProto result = asyncGet.get(-1, null); + if (result.hasDirList()) { + return PBHelperClient.convert(result.getDirList()); + } + return null; + }); + return null; + } + GetListingResponseProto result = ipc(() -> rpcProxy.getListing(null, req)); + if (result.hasDirList()) { + return PBHelperClient.convert(result.getDirList()); + } + return null; + } + + @Override + public BatchedDirectoryListing getBatchedListing( + String[] srcs, byte[] startAfter, boolean needLocation) + throws IOException { + GetBatchedListingRequestProto req = GetBatchedListingRequestProto + .newBuilder() + .addAllPaths(Arrays.asList(srcs)) + .setStartAfter(ByteString.copyFrom(startAfter)) + .setNeedLocation(needLocation).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getBatchedListing(null, req)); + asyncResponse(() -> { + GetBatchedListingResponseProto result = asyncGet.get(-1, null); + + if (result.getListingsCount() > 0) { + HdfsPartialListing[] listingArray = + new HdfsPartialListing[result.getListingsCount()]; + int listingIdx = 0; + for (BatchedDirectoryListingProto proto : result.getListingsList()) { + HdfsPartialListing listing; + if (proto.hasException()) { + HdfsProtos.RemoteExceptionProto reProto = proto.getException(); + RemoteException ex = new RemoteException( + reProto.getClassName(), reProto.getMessage()); + listing = new HdfsPartialListing(proto.getParentIdx(), ex); + } else { + List statuses = + PBHelperClient.convertHdfsFileStatus( + proto.getPartialListingList()); + listing = new HdfsPartialListing(proto.getParentIdx(), statuses); + } + listingArray[listingIdx++] = listing; + } + BatchedDirectoryListing batchedListing = + new BatchedDirectoryListing(listingArray, result.getHasMore(), + result.getStartAfter().toByteArray()); + return batchedListing; + } + return null; + }); + return null; + } + GetBatchedListingResponseProto result = + ipc(() -> rpcProxy.getBatchedListing(null, req)); + + if (result.getListingsCount() > 0) { + HdfsPartialListing[] listingArray = + new HdfsPartialListing[result.getListingsCount()]; + int listingIdx = 0; + for (BatchedDirectoryListingProto proto : result.getListingsList()) { + HdfsPartialListing listing; + if (proto.hasException()) { + HdfsProtos.RemoteExceptionProto reProto = proto.getException(); + RemoteException ex = new RemoteException( + reProto.getClassName(), reProto.getMessage()); + listing = new HdfsPartialListing(proto.getParentIdx(), ex); + } else { + List statuses = + PBHelperClient.convertHdfsFileStatus( + proto.getPartialListingList()); + listing = new HdfsPartialListing(proto.getParentIdx(), statuses); + } + listingArray[listingIdx++] = listing; + } + BatchedDirectoryListing batchedListing = + new BatchedDirectoryListing(listingArray, result.getHasMore(), + result.getStartAfter().toByteArray()); + return batchedListing; + } + return null; + } + + + @Override + public void renewLease(String clientName, List namespaces) + throws IOException { + RenewLeaseRequestProto.Builder builder = RenewLeaseRequestProto + .newBuilder().setClientName(clientName); + if (namespaces != null && !namespaces.isEmpty()) { + builder.addAllNamespaces(namespaces); + } + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.renewLease(null, builder.build())); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.renewLease(null, builder.build())); + } + } + + @Override + public boolean recoverLease(String src, String clientName) + throws IOException { + RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder() + .setSrc(src) + .setClientName(clientName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.recoverLease(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.recoverLease(null, req)).getResult(); + } + + @Override + public long[] getStats() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFsStats(null, VOID_GET_FSSTATUS_REQUEST)); + asyncResponse(() -> PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getFsStats(null, + VOID_GET_FSSTATUS_REQUEST))); + } + + @Override + public ReplicatedBlockStats getReplicatedBlockStats() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getFsReplicatedBlockStats(null, + VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getFsReplicatedBlockStats(null, + VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST))); + } + + @Override + public ECBlockGroupStats getECBlockGroupStats() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFsECBlockGroupStats(null, + VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getFsECBlockGroupStats(null, + VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST))); + } + + @Override + public DatanodeInfo[] getDatanodeReport(DatanodeReportType type) + throws IOException { + GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto + .newBuilder() + .setType(PBHelperClient.convert(type)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDatanodeReport(null, req)); + asyncResponse(() -> + PBHelperClient.convert(asyncGet.get(-1, null).getDiList())); + return null; + } + return PBHelperClient.convert( + ipc(() -> rpcProxy.getDatanodeReport(null, req)).getDiList()); + } + + @Override + public DatanodeStorageReport[] getDatanodeStorageReport( + DatanodeReportType type) throws IOException { + final GetDatanodeStorageReportRequestProto req + = GetDatanodeStorageReportRequestProto.newBuilder() + .setType(PBHelperClient.convert(type)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDatanodeStorageReport(null, req)); + asyncResponse(() -> + PBHelperClient.convertDatanodeStorageReports( + asyncGet.get(-1, null).getDatanodeStorageReportsList())); + return null; + } + return PBHelperClient.convertDatanodeStorageReports( + ipc(() -> rpcProxy.getDatanodeStorageReport(null, req) + .getDatanodeStorageReportsList())); + } + + @Override + public long getPreferredBlockSize(String filename) throws IOException { + GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto + .newBuilder() + .setFilename(filename) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getPreferredBlockSize(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getBsize()); + return -1; + } + return ipc(() -> rpcProxy.getPreferredBlockSize(null, req)).getBsize(); + } + + @Override + public boolean setSafeMode(SafeModeAction action, boolean isChecked) + throws IOException { + SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder() + .setAction(PBHelperClient.convert(action)) + .setChecked(isChecked).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setSafeMode(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.setSafeMode(null, req)).getResult(); + } + + @Override + public boolean saveNamespace(long timeWindow, long txGap) throws IOException { + SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder() + .setTimeWindow(timeWindow).setTxGap(txGap).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.saveNamespace(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getSaved()); + return true; + } + return ipc(() -> rpcProxy.saveNamespace(null, req)).getSaved(); + } + + @Override + public long rollEdits() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rollEdits(null, VOID_ROLLEDITS_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null).getNewSegmentTxId()); + return -1; + } + RollEditsResponseProto resp = ipc(() -> rpcProxy.rollEdits(null, + VOID_ROLLEDITS_REQUEST)); + return resp.getNewSegmentTxId(); + } + + @Override + public boolean restoreFailedStorage(String arg) throws IOException{ + RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto + .newBuilder() + .setArg(arg).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.restoreFailedStorage(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.restoreFailedStorage(null, req)).getResult(); + } + + @Override + public void refreshNodes() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST)); + } + } + + @Override + public void finalizeUpgrade() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST)); + } + } + + @Override + public boolean upgradeStatus() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.upgradeStatus(null, VOID_UPGRADE_STATUS_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null).getUpgradeFinalized()); + return true; + } + final UpgradeStatusResponseProto proto = ipc(() -> rpcProxy.upgradeStatus( + null, VOID_UPGRADE_STATUS_REQUEST)); + return proto.getUpgradeFinalized(); + } + + @Override + public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) + throws IOException { + final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder() + .setAction(PBHelperClient.convert(action)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rollingUpgrade(null, r)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null).getRollingUpgradeInfo())); + return null; + } + final RollingUpgradeResponseProto proto = + ipc(() -> rpcProxy.rollingUpgrade(null, r)); + if (proto.hasRollingUpgradeInfo()) { + return PBHelperClient.convert(proto.getRollingUpgradeInfo()); + } + return null; + } + + @Override + public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie) + throws IOException { + ListCorruptFileBlocksRequestProto.Builder req = + ListCorruptFileBlocksRequestProto.newBuilder().setPath(path); + if (cookie != null) { + req.setCookie(cookie); + } + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listCorruptFileBlocks(null, req.build())); + asyncResponse(() -> asyncGet.get(-1, null).getCorrupt()); + return null; + } + return PBHelperClient.convert( + ipc(() -> rpcProxy.listCorruptFileBlocks(null, req.build())).getCorrupt()); + } + + @Override + public void metaSave(String filename) throws IOException { + MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder() + .setFilename(filename).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.metaSave(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.metaSave(null, req)); + } + } + + @Override + public HdfsFileStatus getFileInfo(String src) throws IOException { + GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder() + .setSrc(src) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFileInfo(null, req)); + asyncResponse(() -> { + GetFileInfoResponseProto res = asyncGet.get(-1, null); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + }); + return null; + } + GetFileInfoResponseProto res = ipc(() -> rpcProxy.getFileInfo(null, req)); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + } + + @Override + public HdfsLocatedFileStatus getLocatedFileInfo(String src, + boolean needBlockToken) throws IOException { + GetLocatedFileInfoRequestProto req = + GetLocatedFileInfoRequestProto.newBuilder() + .setSrc(src) + .setNeedBlockToken(needBlockToken) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getLocatedFileInfo(null, req)); + asyncResponse((AsyncRpcProtocolPBUtil.Response) () -> { + GetLocatedFileInfoResponseProto res = asyncGet.get(-1, null); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + }); + return null; + } + GetLocatedFileInfoResponseProto res = + ipc(() -> rpcProxy.getLocatedFileInfo(null, req)); + return (HdfsLocatedFileStatus) (res.hasFs() + ? PBHelperClient.convert(res.getFs()) + : null); + } + + @Override + public HdfsFileStatus getFileLinkInfo(String src) throws IOException { + GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder() + .setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFileLinkInfo(null, req)); + asyncResponse(() -> { + GetFileLinkInfoResponseProto result = asyncGet.get(-1, null); + return result.hasFs() ? PBHelperClient.convert(result.getFs()) : null; + }); + return null; + } + GetFileLinkInfoResponseProto result = ipc(() -> rpcProxy.getFileLinkInfo(null, req)); + return result.hasFs() ? PBHelperClient.convert(result.getFs()) : null; + } + + @Override + public ContentSummary getContentSummary(String path) throws IOException { + GetContentSummaryRequestProto req = GetContentSummaryRequestProto + .newBuilder() + .setPath(path) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getContentSummary(null, req)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null).getSummary())); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getContentSummary(null, req)) + .getSummary()); + } + + @Override + public void setQuota(String path, long namespaceQuota, long storagespaceQuota, + StorageType type) throws IOException { + final SetQuotaRequestProto.Builder builder + = SetQuotaRequestProto.newBuilder() + .setPath(path) + .setNamespaceQuota(namespaceQuota) + .setStoragespaceQuota(storagespaceQuota); + if (type != null) { + builder.setStorageType(PBHelperClient.convertStorageType(type)); + } + final SetQuotaRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setQuota(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.setQuota(null, req)); + } + } + + @Override + public void fsync(String src, long fileId, String client, + long lastBlockLength) throws IOException { + FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src) + .setClient(client).setLastBlockLength(lastBlockLength) + .setFileId(fileId).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.fsync(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.fsync(null, req)); + } + } + + @Override + public void setTimes(String src, long mtime, long atime) throws IOException { + SetTimesRequestProto req = SetTimesRequestProto.newBuilder() + .setSrc(src) + .setMtime(mtime) + .setAtime(atime) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setTimes(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.setTimes(null, req)); + } + } + + @Override + public void createSymlink(String target, String link, FsPermission dirPerm, + boolean createParent) throws IOException { + CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder() + .setTarget(target) + .setLink(link) + .setDirPerm(PBHelperClient.convert(dirPerm)) + .setCreateParent(createParent) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.createSymlink(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.createSymlink(null, req)); + } + } + + @Override + public String getLinkTarget(String path) throws IOException { + GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder() + .setPath(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getLinkTarget(null, req)); + asyncResponse(new AsyncRpcProtocolPBUtil.Response() { + @Override + public Object response() throws Exception { + GetLinkTargetResponseProto rsp = asyncGet.get(-1, null); + return rsp.hasTargetPath() ? rsp.getTargetPath() : null; + } + }); + return null; + } + GetLinkTargetResponseProto rsp = ipc(() -> rpcProxy.getLinkTarget(null, req)); + return rsp.hasTargetPath() ? rsp.getTargetPath() : null; + } + + @Override + public LocatedBlock updateBlockForPipeline(ExtendedBlock block, + String clientName) throws IOException { + UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto + .newBuilder() + .setBlock(PBHelperClient.convert(block)) + .setClientName(clientName) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.updateBlockForPipeline(null, req)); + asyncResponse( + () -> PBHelperClient.convertLocatedBlockProto( + asyncGet.get(-1, null).getBlock())); + return null; + } + return PBHelperClient.convertLocatedBlockProto( + ipc(() -> rpcProxy.updateBlockForPipeline(null, req)).getBlock()); + } + + @Override + public void updatePipeline(String clientName, ExtendedBlock oldBlock, + ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) + throws IOException { + UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder() + .setClientName(clientName) + .setOldBlock(PBHelperClient.convert(oldBlock)) + .setNewBlock(PBHelperClient.convert(newBlock)) + .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes))) + .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.updatePipeline(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.updatePipeline(null, req)); + } + } + + @Override + public Token getDelegationToken(Text renewer) + throws IOException { + GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto + .newBuilder() + .setRenewer(renewer == null ? "" : renewer.toString()) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDelegationToken(null, req)); + asyncResponse(() -> { + GetDelegationTokenResponseProto resp = asyncGet.get(-1, null); + return resp.hasToken() ? + PBHelperClient.convertDelegationToken(resp.getToken()) : null; + }); + return null; + } + GetDelegationTokenResponseProto resp = + ipc(() -> rpcProxy.getDelegationToken(null, req)); + return resp.hasToken() ? + PBHelperClient.convertDelegationToken(resp.getToken()) : null; + } + + @Override + public long renewDelegationToken(Token token) + throws IOException { + RenewDelegationTokenRequestProto req = + RenewDelegationTokenRequestProto.newBuilder(). + setToken(PBHelperClient.convert(token)). + build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.renewDelegationToken(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getNewExpiryTime()); + return -1; + } + return ipc(() -> rpcProxy.renewDelegationToken(null, req)).getNewExpiryTime(); + } + + @Override + public void cancelDelegationToken(Token token) + throws IOException { + CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto + .newBuilder() + .setToken(PBHelperClient.convert(token)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.cancelDelegationToken(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.cancelDelegationToken(null, req)); + } + } + + @Override + public void setBalancerBandwidth(long bandwidth) throws IOException { + SetBalancerBandwidthRequestProto req = + SetBalancerBandwidthRequestProto.newBuilder() + .setBandwidth(bandwidth) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setBalancerBandwidth(null, req)); + asyncResponse(new Response() { + @Override + public Object response() throws Exception { + asyncGet.get(-1, null); + return null; + } + }); + } else { + ipc(() -> rpcProxy.setBalancerBandwidth(null, req)); + } + } + + @Override + public boolean isMethodSupported(String methodName) throws IOException { + return RpcClientUtil.isMethodSupported(rpcProxy, + ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER, + RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName); + } + + @Override + public DataEncryptionKey getDataEncryptionKey() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDataEncryptionKey(null, + VOID_GET_DATA_ENCRYPTIONKEY_REQUEST)); + asyncResponse(() -> { + GetDataEncryptionKeyResponseProto rsp = asyncGet.get(-1, null); + return rsp.hasDataEncryptionKey() ? + PBHelperClient.convert(rsp.getDataEncryptionKey()) : null; + }); + return null; + } + GetDataEncryptionKeyResponseProto rsp = ipc(() -> rpcProxy.getDataEncryptionKey( + null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST)); + return rsp.hasDataEncryptionKey() ? + PBHelperClient.convert(rsp.getDataEncryptionKey()) : null; + } + + + @Override + public boolean isFileClosed(String src) throws IOException { + IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder() + .setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.isFileClosed(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.isFileClosed(null, req)).getResult(); + } + + @Override + public Object getUnderlyingProxyObject() { + return rpcProxy; + } + + @Override + public String createSnapshot(String snapshotRoot, String snapshotName) + throws IOException { + final CreateSnapshotRequestProto.Builder builder + = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot); + if (snapshotName != null) { + builder.setSnapshotName(snapshotName); + } + final CreateSnapshotRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.createSnapshot(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getSnapshotPath()); + return null; + } + return ipc(() -> rpcProxy.createSnapshot(null, req)).getSnapshotPath(); + } + + @Override + public void deleteSnapshot(String snapshotRoot, String snapshotName) + throws IOException { + DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.deleteSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.deleteSnapshot(null, req)); + } + } + + @Override + public void allowSnapshot(String snapshotRoot) throws IOException { + AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.allowSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.allowSnapshot(null, req)); + } + } + + @Override + public void disallowSnapshot(String snapshotRoot) throws IOException { + DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto + .newBuilder().setSnapshotRoot(snapshotRoot).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.disallowSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.disallowSnapshot(null, req)); + } + } + + @Override + public void renameSnapshot(String snapshotRoot, String snapshotOldName, + String snapshotNewName) throws IOException { + RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName) + .setSnapshotNewName(snapshotNewName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.renameSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.renameSnapshot(null, req)); + } + } + + @Override + public SnapshottableDirectoryStatus[] getSnapshottableDirListing() + throws IOException { + GetSnapshottableDirListingRequestProto req = + GetSnapshottableDirListingRequestProto.newBuilder().build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshottableDirListing(null, req)); + asyncResponse(() -> { + GetSnapshottableDirListingResponseProto result = asyncGet.get(-1, null); + if (result.hasSnapshottableDirList()) { + return PBHelperClient.convert(result.getSnapshottableDirList()); + } + return null; + }); + return null; + } + GetSnapshottableDirListingResponseProto result = ipc(() -> rpcProxy + .getSnapshottableDirListing(null, req)); + + if (result.hasSnapshottableDirList()) { + return PBHelperClient.convert(result.getSnapshottableDirList()); + } + return null; + } + + @Override + public SnapshotStatus[] getSnapshotListing(String path) + throws IOException { + GetSnapshotListingRequestProto req = + GetSnapshotListingRequestProto.newBuilder() + .setSnapshotRoot(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshotListing(null, req)); + asyncResponse(() -> { + GetSnapshotListingResponseProto result = asyncGet.get(-1, null); + if (result.hasSnapshotList()) { + return PBHelperClient.convert(result.getSnapshotList()); + } + return null; + }); + return null; + } + GetSnapshotListingResponseProto result = ipc(() -> rpcProxy + .getSnapshotListing(null, req)); + + if (result.hasSnapshotList()) { + return PBHelperClient.convert(result.getSnapshotList()); + } + return null; + } + + @Override + public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot, + String fromSnapshot, String toSnapshot) throws IOException { + GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto + .newBuilder().setSnapshotRoot(snapshotRoot) + .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshotDiffReport(null, req)); + asyncResponse(() -> PBHelperClient.convert(asyncGet.get(-1, null) + .getDiffReport())); + return null; + } + GetSnapshotDiffReportResponseProto result = + ipc(() -> rpcProxy.getSnapshotDiffReport(null, req)); + + return PBHelperClient.convert(result.getDiffReport()); + } + + @Override + public SnapshotDiffReportListing getSnapshotDiffReportListing( + String snapshotRoot, String fromSnapshot, String toSnapshot, + byte[] startPath, int index) throws IOException { + GetSnapshotDiffReportListingRequestProto req = + GetSnapshotDiffReportListingRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setFromSnapshot(fromSnapshot) + .setToSnapshot(toSnapshot).setCursor( + HdfsProtos.SnapshotDiffReportCursorProto.newBuilder() + .setStartPath(PBHelperClient.getByteString(startPath)) + .setIndex(index).build()).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshotDiffReportListing(null, req)); + asyncResponse(new Response() { + @Override + public Object response() throws Exception { + return PBHelperClient.convert(asyncGet.get(-1, null).getDiffReport()); + } + }); + return null; + } + GetSnapshotDiffReportListingResponseProto result = + ipc(() -> rpcProxy.getSnapshotDiffReportListing(null, req)); + + return PBHelperClient.convert(result.getDiffReport()); + } + + @Override + public long addCacheDirective(CacheDirectiveInfo directive, + EnumSet flags) throws IOException { + AddCacheDirectiveRequestProto.Builder builder = + AddCacheDirectiveRequestProto.newBuilder(). + setInfo(PBHelperClient.convert(directive)); + if (!flags.isEmpty()) { + builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags)); + } + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.addCacheDirective(null, builder.build())); + asyncResponse(() -> asyncGet.get(-1, null).getId()); + return -1; + } + return ipc(() -> rpcProxy.addCacheDirective(null, builder.build())).getId(); + } + + @Override + public void modifyCacheDirective(CacheDirectiveInfo directive, + EnumSet flags) throws IOException { + ModifyCacheDirectiveRequestProto.Builder builder = + ModifyCacheDirectiveRequestProto.newBuilder(). + setInfo(PBHelperClient.convert(directive)); + if (!flags.isEmpty()) { + builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags)); + } + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.modifyCacheDirective(null, builder.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.modifyCacheDirective(null, builder.build())); + } + } + + @Override + public void removeCacheDirective(long id) + throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeCacheDirective(null, + RemoveCacheDirectiveRequestProto.newBuilder(). + setId(id).build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeCacheDirective(null, + RemoveCacheDirectiveRequestProto.newBuilder(). + setId(id).build())); + } + } + + private static class BatchedCacheEntries + implements BatchedEntries { + private final ListCacheDirectivesResponseProto response; + + BatchedCacheEntries( + ListCacheDirectivesResponseProto response) { + this.response = response; + } + + @Override + public CacheDirectiveEntry get(int i) { + return PBHelperClient.convert(response.getElements(i)); + } + + @Override + public int size() { + return response.getElementsCount(); + } + + @Override + public boolean hasMore() { + return response.getHasMore(); + } + } + + @Override + public BatchedEntries listCacheDirectives(long prevId, + CacheDirectiveInfo filter) throws IOException { + if (filter == null) { + filter = new CacheDirectiveInfo.Builder().build(); + } + CacheDirectiveInfo f = filter; + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.listCacheDirectives(null, + ListCacheDirectivesRequestProto.newBuilder(). + setPrevId(prevId). + setFilter(PBHelperClient.convert(f)). + build())); + asyncResponse(() -> new BatchedCacheEntries(asyncGet.get(-1, null))); + return null; + } + return new BatchedCacheEntries( + ipc(() -> rpcProxy.listCacheDirectives(null, + ListCacheDirectivesRequestProto.newBuilder(). + setPrevId(prevId). + setFilter(PBHelperClient.convert(f)). + build()))); + } + + @Override + public void addCachePool(CachePoolInfo info) throws IOException { + AddCachePoolRequestProto.Builder builder = + AddCachePoolRequestProto.newBuilder(); + builder.setInfo(PBHelperClient.convert(info)); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.addCachePool(null, builder.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.addCachePool(null, builder.build())); + } + } + + @Override + public void modifyCachePool(CachePoolInfo req) throws IOException { + ModifyCachePoolRequestProto.Builder builder = + ModifyCachePoolRequestProto.newBuilder(); + builder.setInfo(PBHelperClient.convert(req)); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.modifyCachePool(null, builder.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.modifyCachePool(null, builder.build())); + } + } + + @Override + public void removeCachePool(String cachePoolName) throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeCachePool(null, + RemoveCachePoolRequestProto.newBuilder(). + setPoolName(cachePoolName).build())); + + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeCachePool(null, + RemoveCachePoolRequestProto.newBuilder(). + setPoolName(cachePoolName).build())); + } + } + + private static class BatchedCachePoolEntries + implements BatchedEntries { + private final ListCachePoolsResponseProto proto; + + public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) { + this.proto = proto; + } + + @Override + public CachePoolEntry get(int i) { + CachePoolEntryProto elem = proto.getEntries(i); + return PBHelperClient.convert(elem); + } + + @Override + public int size() { + return proto.getEntriesCount(); + } + + @Override + public boolean hasMore() { + return proto.getHasMore(); + } + } + + @Override + public BatchedEntries listCachePools(String prevKey) + throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listCachePools(null, + ListCachePoolsRequestProto.newBuilder(). + setPrevPoolName(prevKey).build())); + asyncResponse((Response) () -> + new BatchedCachePoolEntries(asyncGet.get(-1, null))); + } + return new BatchedCachePoolEntries( + ipc(() -> rpcProxy.listCachePools(null, + ListCachePoolsRequestProto.newBuilder(). + setPrevPoolName(prevKey).build()))); + } + + @Override + public void modifyAclEntries(String src, List aclSpec) + throws IOException { + ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto + .newBuilder().setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.modifyAclEntries(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.modifyAclEntries(null, req)); + } + } + + @Override + public void removeAclEntries(String src, List aclSpec) + throws IOException { + RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto + .newBuilder().setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeAclEntries(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeAclEntries(null, req)); + } + } + + @Override + public void removeDefaultAcl(String src) throws IOException { + RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto + .newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeDefaultAcl(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeDefaultAcl(null, req)); + } + } + + @Override + public void removeAcl(String src) throws IOException { + RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder() + .setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeAcl(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeAcl(null, req)); + } + } + + @Override + public void setAcl(String src, List aclSpec) throws IOException { + SetAclRequestProto req = SetAclRequestProto.newBuilder() + .setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setAcl(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setAcl(null, req)); + } + } + + @Override + public AclStatus getAclStatus(String src) throws IOException { + GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder() + .setSrc(src).build(); + try { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getAclStatus(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } else { + return PBHelperClient.convert(rpcProxy.getAclStatus(null, req)); + } + } catch (ServiceException e) { + throw getRemoteException(e); + } + } + + @Override + public void createEncryptionZone(String src, String keyName) + throws IOException { + final CreateEncryptionZoneRequestProto.Builder builder = + CreateEncryptionZoneRequestProto.newBuilder(); + builder.setSrc(src); + if (keyName != null && !keyName.isEmpty()) { + builder.setKeyName(keyName); + } + CreateEncryptionZoneRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.createEncryptionZone(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.createEncryptionZone(null, req)); + } + } + + @Override + public EncryptionZone getEZForPath(String src) throws IOException { + final GetEZForPathRequestProto.Builder builder = + GetEZForPathRequestProto.newBuilder(); + builder.setSrc(src); + final GetEZForPathRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getEZForPath(null, req)); + asyncResponse((Response) () -> { + final EncryptionZonesProtos.GetEZForPathResponseProto response + = asyncGet.get(-1, null); + if (response.hasZone()) { + return PBHelperClient.convert(response.getZone()); + } else { + return null; + } + }); + return null; + } + final EncryptionZonesProtos.GetEZForPathResponseProto response = + ipc(() -> rpcProxy.getEZForPath(null, req)); + if (response.hasZone()) { + return PBHelperClient.convert(response.getZone()); + } else { + return null; + } + } + + @Override + public BatchedEntries listEncryptionZones(long id) + throws IOException { + final ListEncryptionZonesRequestProto req = + ListEncryptionZonesRequestProto.newBuilder() + .setId(id) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listEncryptionZones(null, req)); + asyncResponse((Response) () -> { + EncryptionZonesProtos.ListEncryptionZonesResponseProto response + = asyncGet.get(-1, null); + List elements = + Lists.newArrayListWithCapacity(response.getZonesCount()); + for (EncryptionZoneProto p : response.getZonesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + }); + return null; + } + EncryptionZonesProtos.ListEncryptionZonesResponseProto response = + ipc(() -> rpcProxy.listEncryptionZones(null, req)); + List elements = + Lists.newArrayListWithCapacity(response.getZonesCount()); + for (EncryptionZoneProto p : response.getZonesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + } + + @Override + public void setErasureCodingPolicy(String src, String ecPolicyName) + throws IOException { + final SetErasureCodingPolicyRequestProto.Builder builder = + SetErasureCodingPolicyRequestProto.newBuilder(); + builder.setSrc(src); + if (ecPolicyName != null) { + builder.setEcPolicyName(ecPolicyName); + } + SetErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setErasureCodingPolicy(null, req)); + } + } + + @Override + public void unsetErasureCodingPolicy(String src) throws IOException { + final UnsetErasureCodingPolicyRequestProto.Builder builder = + UnsetErasureCodingPolicyRequestProto.newBuilder(); + builder.setSrc(src); + UnsetErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.unsetErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.unsetErasureCodingPolicy(null, req)); + } + } + + @Override + public ECTopologyVerifierResult getECTopologyResultForPolicies( + final String... policyNames) throws IOException { + final GetECTopologyResultForPoliciesRequestProto.Builder builder = + GetECTopologyResultForPoliciesRequestProto.newBuilder(); + builder.addAllPolicies(Arrays.asList(policyNames)); + GetECTopologyResultForPoliciesRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getECTopologyResultForPolicies(null, req)); + asyncResponse((Response) () -> PBHelperClient + .convertECTopologyVerifierResultProto( + asyncGet.get(-1, null).getResponse())); + } + GetECTopologyResultForPoliciesResponseProto response = + ipc(() -> rpcProxy.getECTopologyResultForPolicies(null, req)); + return PBHelperClient + .convertECTopologyVerifierResultProto(response.getResponse()); + } + + @Override + public void reencryptEncryptionZone(String zone, ReencryptAction action) + throws IOException { + final ReencryptEncryptionZoneRequestProto.Builder builder = + ReencryptEncryptionZoneRequestProto.newBuilder(); + builder.setZone(zone).setAction(PBHelperClient.convert(action)); + ReencryptEncryptionZoneRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.reencryptEncryptionZone(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.reencryptEncryptionZone(null, req)); + } + } + + @Override + public BatchedEntries listReencryptionStatus(long id) + throws IOException { + final ListReencryptionStatusRequestProto req = + ListReencryptionStatusRequestProto.newBuilder().setId(id).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listReencryptionStatus(null, req)); + asyncResponse((Response) () -> { + ListReencryptionStatusResponseProto response = asyncGet.get(-1, null); + List elements = + Lists.newArrayListWithCapacity(response.getStatusesCount()); + for (ZoneReencryptionStatusProto p : response.getStatusesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + }); + return null; + } + ListReencryptionStatusResponseProto response = + ipc(() -> rpcProxy.listReencryptionStatus(null, req)); + List elements = + Lists.newArrayListWithCapacity(response.getStatusesCount()); + for (ZoneReencryptionStatusProto p : response.getStatusesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + } + + @Override + public void setXAttr(String src, XAttr xAttr, EnumSet flag) + throws IOException { + SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder() + .setSrc(src) + .setXAttr(PBHelperClient.convertXAttrProto(xAttr)) + .setFlag(PBHelperClient.convert(flag)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setXAttr(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setXAttr(null, req)); + } + } + + @Override + public List getXAttrs(String src, List xAttrs) + throws IOException { + GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder(); + builder.setSrc(src); + if (xAttrs != null) { + builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs)); + } + GetXAttrsRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getXAttrs(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getXAttrs(null, req))); + } + + @Override + public List listXAttrs(String src) throws IOException { + ListXAttrsRequestProto.Builder builder = + ListXAttrsRequestProto.newBuilder(); + builder.setSrc(src); + ListXAttrsRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listXAttrs(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.listXAttrs(null, req))); + } + + @Override + public void removeXAttr(String src, XAttr xAttr) throws IOException { + RemoveXAttrRequestProto req = RemoveXAttrRequestProto + .newBuilder().setSrc(src) + .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeXAttr(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeXAttr(null, req)); + } + } + + @Override + public void checkAccess(String path, FsAction mode) throws IOException { + CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder() + .setPath(path).setMode(PBHelperClient.convert(mode)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.checkAccess(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.checkAccess(null, req)); + } + } + + @Override + public void setStoragePolicy(String src, String policyName) + throws IOException { + SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto + .newBuilder().setSrc(src).setPolicyName(policyName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setStoragePolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setStoragePolicy(null, req)); + } + } + + @Override + public void unsetStoragePolicy(String src) throws IOException { + UnsetStoragePolicyRequestProto req = UnsetStoragePolicyRequestProto + .newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.unsetStoragePolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.unsetStoragePolicy(null, req)); + } + } + + @Override + public BlockStoragePolicy getStoragePolicy(String path) throws IOException { + GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto + .newBuilder().setPath(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getStoragePolicy(null, request)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null).getStoragePolicy())); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getStoragePolicy(null, request)) + .getStoragePolicy()); + } + + @Override + public BlockStoragePolicy[] getStoragePolicies() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy + .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST)); + asyncResponse((Response) () -> PBHelperClient.convertStoragePolicies( + asyncGet.get(-1, null).getPoliciesList())); + return null; + } + GetStoragePoliciesResponseProto response = ipc(() -> rpcProxy + .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST)); + return PBHelperClient.convertStoragePolicies(response.getPoliciesList()); + } + + public long getCurrentEditLogTxid() throws IOException { + GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto + .getDefaultInstance(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getCurrentEditLogTxid(null, req)); + asyncResponse((Response) () -> asyncGet.get(-1, null).getTxid()); + return -1; + } + return ipc(() -> rpcProxy.getCurrentEditLogTxid(null, req)).getTxid(); + } + + @Override + public EventBatchList getEditsFromTxid(long txid) throws IOException { + GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder() + .setTxid(txid).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getEditsFromTxid(null, req)); + asyncResponse((Response) () -> PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getEditsFromTxid(null, req))); + } + + @Override + public AddErasureCodingPolicyResponse[] addErasureCodingPolicies( + ErasureCodingPolicy[] policies) throws IOException { + List protos = Arrays.stream(policies) + .map(PBHelperClient::convertErasureCodingPolicy) + .collect(Collectors.toList()); + AddErasureCodingPoliciesRequestProto req = + AddErasureCodingPoliciesRequestProto.newBuilder() + .addAllEcPolicies(protos).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy + .addErasureCodingPolicies(null, req)); + asyncResponse((Response) () -> { + AddErasureCodingPoliciesResponseProto rep = asyncGet.get(-1, null); + AddErasureCodingPolicyResponse[] responses = + rep.getResponsesList().stream() + .map(PBHelperClient::convertAddErasureCodingPolicyResponse) + .toArray(AddErasureCodingPolicyResponse[]::new); + return responses; + }); + return null; + } + AddErasureCodingPoliciesResponseProto rep = ipc(() -> rpcProxy + .addErasureCodingPolicies(null, req)); + AddErasureCodingPolicyResponse[] responses = + rep.getResponsesList().stream() + .map(PBHelperClient::convertAddErasureCodingPolicyResponse) + .toArray(AddErasureCodingPolicyResponse[]::new); + return responses; + } + + @Override + public void removeErasureCodingPolicy(String ecPolicyName) + throws IOException { + RemoveErasureCodingPolicyRequestProto.Builder builder = + RemoveErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + RemoveErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeErasureCodingPolicy(null, req)); + } + } + + @Override + public void enableErasureCodingPolicy(String ecPolicyName) + throws IOException { + EnableErasureCodingPolicyRequestProto.Builder builder = + EnableErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + EnableErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.enableErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.enableErasureCodingPolicy(null, req)); + } + } + + @Override + public void disableErasureCodingPolicy(String ecPolicyName) + throws IOException { + DisableErasureCodingPolicyRequestProto.Builder builder = + DisableErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + DisableErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.disableErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.disableErasureCodingPolicy(null, req)); + } + } + + @Override + public ErasureCodingPolicyInfo[] getErasureCodingPolicies() + throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy + .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST)); + asyncResponse((Response) () -> { + GetErasureCodingPoliciesResponseProto response = asyncGet.get(-1, null); + ErasureCodingPolicyInfo[] ecPolicies = + new ErasureCodingPolicyInfo[response.getEcPoliciesCount()]; + int i = 0; + for (ErasureCodingPolicyProto proto : response.getEcPoliciesList()) { + ecPolicies[i++] = + PBHelperClient.convertErasureCodingPolicyInfo(proto); + } + return ecPolicies; + }); + return null; + } + GetErasureCodingPoliciesResponseProto response = ipc(() -> rpcProxy + .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST)); + ErasureCodingPolicyInfo[] ecPolicies = + new ErasureCodingPolicyInfo[response.getEcPoliciesCount()]; + int i = 0; + for (ErasureCodingPolicyProto proto : response.getEcPoliciesList()) { + ecPolicies[i++] = + PBHelperClient.convertErasureCodingPolicyInfo(proto); + } + return ecPolicies; + } + + @Override + public Map getErasureCodingCodecs() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy + .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST)); + asyncResponse(() -> { + GetErasureCodingCodecsResponseProto response = asyncGet.get(-1, null); + Map ecCodecs = new HashMap<>(); + for (CodecProto codec : response.getCodecList()) { + ecCodecs.put(codec.getCodec(), codec.getCoders()); + } + return ecCodecs; + }); + return null; + } + GetErasureCodingCodecsResponseProto response = ipc(() -> rpcProxy + .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST)); + Map ecCodecs = new HashMap<>(); + for (CodecProto codec : response.getCodecList()) { + ecCodecs.put(codec.getCodec(), codec.getCoders()); + } + return ecCodecs; + } + + @Override + public ErasureCodingPolicy getErasureCodingPolicy(String src) + throws IOException { + GetErasureCodingPolicyRequestProto req = + GetErasureCodingPolicyRequestProto.newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getErasureCodingPolicy(null, req)); + asyncResponse((Response) () -> { + GetErasureCodingPolicyResponseProto response = asyncGet.get(-1, null); + if (response.hasEcPolicy()) { + return PBHelperClient.convertErasureCodingPolicy( + response.getEcPolicy()); + } + return null; + }); + return null; + } + GetErasureCodingPolicyResponseProto response = + ipc(() -> rpcProxy.getErasureCodingPolicy(null, req)); + if (response.hasEcPolicy()) { + return PBHelperClient.convertErasureCodingPolicy( + response.getEcPolicy()); + } + return null; + } + + @Override + public QuotaUsage getQuotaUsage(String path) throws IOException { + GetQuotaUsageRequestProto req = + GetQuotaUsageRequestProto.newBuilder().setPath(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getQuotaUsage(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null).getUsage())); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getQuotaUsage(null, req)) + .getUsage()); + } + + @Deprecated + @Override + public BatchedEntries listOpenFiles(long prevId) + throws IOException { + return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES), + OpenFilesIterator.FILTER_PATH_DEFAULT); + } + + @Override + public BatchedEntries listOpenFiles(long prevId, + EnumSet openFilesTypes, String path) throws IOException { + ListOpenFilesRequestProto.Builder req = + ListOpenFilesRequestProto.newBuilder().setId(prevId); + if (openFilesTypes != null) { + req.addAllTypes(PBHelperClient.convertOpenFileTypes(openFilesTypes)); + } + req.setPath(path); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listOpenFiles(null, req.build())); + asyncResponse(() -> { + ListOpenFilesResponseProto response = asyncGet.get(-1, null); + List openFileEntries = + Lists.newArrayListWithCapacity(response.getEntriesCount()); + for (OpenFilesBatchResponseProto p : response.getEntriesList()) { + openFileEntries.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(openFileEntries, response.getHasMore()); + }); + return null; + } + ListOpenFilesResponseProto response = + ipc(() -> rpcProxy.listOpenFiles(null, req.build())); + List openFileEntries = + Lists.newArrayListWithCapacity(response.getEntriesCount()); + for (OpenFilesBatchResponseProto p : response.getEntriesList()) { + openFileEntries.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(openFileEntries, response.getHasMore()); + } + + @Override + public void msync() throws IOException { + MsyncRequestProto.Builder req = MsyncRequestProto.newBuilder(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.msync(null, req.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.msync(null, req.build())); + } + } + + @Override + public void satisfyStoragePolicy(String src) throws IOException { + SatisfyStoragePolicyRequestProto req = + SatisfyStoragePolicyRequestProto.newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.satisfyStoragePolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.satisfyStoragePolicy(null, req)); + } + } + + @Override + public DatanodeInfo[] getSlowDatanodeReport() throws IOException { + GetSlowDatanodeReportRequestProto req = + GetSlowDatanodeReportRequestProto.newBuilder().build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSlowDatanodeReport(null, req)); + asyncResponse(() -> + PBHelperClient.convert(asyncGet.get(-1, null).getDatanodeInfoProtoList())); + return null; + } + return PBHelperClient.convert( + ipc(() -> rpcProxy.getSlowDatanodeReport(null, req)).getDatanodeInfoProtoList()); + } + + @Override + public HAServiceProtocol.HAServiceState getHAServiceState() + throws IOException { + HAServiceStateRequestProto req = + HAServiceStateRequestProto.newBuilder().build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getHAServiceState(null, req)); + asyncResponse(() -> { + HAServiceStateProto res = asyncGet.get(-1, null).getState(); + switch(res) { + case ACTIVE: + return HAServiceProtocol.HAServiceState.ACTIVE; + case STANDBY: + return HAServiceProtocol.HAServiceState.STANDBY; + case OBSERVER: + return HAServiceProtocol.HAServiceState.OBSERVER; + case INITIALIZING: + default: + return HAServiceProtocol.HAServiceState.INITIALIZING; + } + }); + return null; + } + HAServiceStateProto res = + ipc(() -> rpcProxy.getHAServiceState(null, req)).getState(); + switch(res) { + case ACTIVE: + return HAServiceProtocol.HAServiceState.ACTIVE; + case STANDBY: + return HAServiceProtocol.HAServiceState.STANDBY; + case OBSERVER: + return HAServiceProtocol.HAServiceState.OBSERVER; + case INITIALIZING: + default: + return HAServiceProtocol.HAServiceState.INITIALIZING; + } + } + + @Override + public Path getEnclosingRoot(String filename) throws IOException { + final GetEnclosingRootRequestProto.Builder builder = + GetEnclosingRootRequestProto.newBuilder(); + builder.setFilename(filename); + final GetEnclosingRootRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getEnclosingRoot(null, req)); + asyncResponse(() -> + new Path(asyncGet.get(-1, null).getEnclosingRootPath())); + return null; + } + try { + final GetEnclosingRootResponseProto response = + rpcProxy.getEnclosingRoot(null, req); + return new Path(response.getEnclosingRootPath()); + } catch (ServiceException e) { + throw getRemoteException(e); + } + } +} + diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolServerSideTranslatorPB.java new file mode 100644 index 0000000000000..003c4a5607c55 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolServerSideTranslatorPB.java @@ -0,0 +1,43 @@ +package org.apache.hadoop.hdfs.protocolPB; + + +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +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.thirdparty.protobuf.RpcController; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; +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; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolTranslatorPB.java new file mode 100644 index 0000000000000..6958e153696c7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolTranslatorPB.java @@ -0,0 +1,38 @@ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos; +import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB; +import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB; +import org.apache.hadoop.util.concurrent.AsyncGet; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; + +public class RouterGetUserMappingsProtocolTranslatorPB + extends GetUserMappingsProtocolClientSideTranslatorPB { + private final GetUserMappingsProtocolPB rpcProxy; + + public RouterGetUserMappingsProtocolTranslatorPB(GetUserMappingsProtocolPB rpcProxy) { + super(rpcProxy); + this.rpcProxy = rpcProxy; + } + + @Override + public String[] getGroupsForUser(String user) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getGroupsForUser(user); + } + GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto request = GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto + .newBuilder().setUser(user).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getGroupsForUser(NULL_CONTROLLER, request)); + asyncResponse(() -> { + GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto resp = asyncGet.get(-1, null); + return resp.getGroupsList().toArray(new String[resp.getGroupsCount()]); + }); + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolServerSideTranslatorPB.java new file mode 100644 index 0000000000000..a015aee05da0c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolServerSideTranslatorPB.java @@ -0,0 +1,243 @@ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.namenode.NNStorage; +import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; +import org.apache.hadoop.thirdparty.protobuf.RpcController; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncRouterServer; + +public class RouterNamenodeProtocolServerSideTranslatorPB + extends NamenodeProtocolServerSideTranslatorPB{ + private final RouterRpcServer server; + private final boolean isAsyncRpc; + + public RouterNamenodeProtocolServerSideTranslatorPB(NamenodeProtocol impl) { + super(impl); + this.server = (RouterRpcServer) impl; + this.isAsyncRpc = server.isAsync(); + } + + + @Override + public NamenodeProtocolProtos.GetBlocksResponseProto getBlocks( + RpcController unused, + NamenodeProtocolProtos.GetBlocksRequestProto request) { + if (!isAsyncRpc) { + return getBlocks(unused, request); + } + asyncRouterServer(() -> { + DatanodeInfo dnInfo = new DatanodeInfo.DatanodeInfoBuilder() + .setNodeID(PBHelperClient.convert(request.getDatanode())) + .build(); + return server.getBlocks(dnInfo, request.getSize(), + request.getMinBlockSize(), request.getTimeInterval(), + request.hasStorageType() ? + PBHelperClient.convertStorageType(request.getStorageType()) : null); + }, blocks -> + NamenodeProtocolProtos.GetBlocksResponseProto.newBuilder() + .setBlocks(PBHelper.convert(blocks)).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.GetBlockKeysResponseProto getBlockKeys( + RpcController unused, + NamenodeProtocolProtos.GetBlockKeysRequestProto request) { + if (!isAsyncRpc) { + return getBlockKeys(unused, request); + } + asyncRouterServer(server::getBlockKeys, keys -> { + NamenodeProtocolProtos.GetBlockKeysResponseProto.Builder builder = + NamenodeProtocolProtos.GetBlockKeysResponseProto.newBuilder(); + if (keys != null) { + builder.setKeys(PBHelper.convert(keys)); + } + return builder.build(); + }); + return null; + } + + @Override + public NamenodeProtocolProtos.GetTransactionIdResponseProto getTransactionId( + RpcController unused, + NamenodeProtocolProtos.GetTransactionIdRequestProto request) { + if (!isAsyncRpc) { + return getTransactionId(unused, request); + } + asyncRouterServer(server::getTransactionID, + txid -> NamenodeProtocolProtos + .GetTransactionIdResponseProto + .newBuilder().setTxId(txid).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.GetMostRecentCheckpointTxIdResponseProto getMostRecentCheckpointTxId( + RpcController unused, NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto request) { + if (!isAsyncRpc) { + return getMostRecentCheckpointTxId(unused, request); + } + asyncRouterServer(server::getMostRecentCheckpointTxId, + txid -> NamenodeProtocolProtos + .GetMostRecentCheckpointTxIdResponseProto + .newBuilder().setTxId(txid).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.GetMostRecentNameNodeFileTxIdResponseProto getMostRecentNameNodeFileTxId( + RpcController unused, NamenodeProtocolProtos.GetMostRecentNameNodeFileTxIdRequestProto request) { + if (!isAsyncRpc) { + return getMostRecentNameNodeFileTxId(unused, request); + } + asyncRouterServer(() -> server.getMostRecentNameNodeFileTxId( + NNStorage.NameNodeFile.valueOf(request.getNameNodeFile())), + txid -> NamenodeProtocolProtos + .GetMostRecentNameNodeFileTxIdResponseProto + .newBuilder().setTxId(txid).build()); + return null; + } + + + @Override + public NamenodeProtocolProtos.RollEditLogResponseProto rollEditLog( + RpcController unused, + NamenodeProtocolProtos.RollEditLogRequestProto request) { + if (!isAsyncRpc) { + return rollEditLog(unused, request); + } + asyncRouterServer(server::rollEditLog, + signature -> NamenodeProtocolProtos + .RollEditLogResponseProto.newBuilder() + .setSignature(PBHelper.convert(signature)).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.ErrorReportResponseProto errorReport( + RpcController unused, + NamenodeProtocolProtos.ErrorReportRequestProto request) { + if (!isAsyncRpc) { + return errorReport(unused, request); + } + asyncRouterServer(() -> { + server.errorReport(PBHelper.convert(request.getRegistration()), + request.getErrorCode(), request.getMsg()); + return null; + }, result -> VOID_ERROR_REPORT_RESPONSE); + return null; + } + + @Override + public NamenodeProtocolProtos.RegisterResponseProto registerSubordinateNamenode( + RpcController unused, NamenodeProtocolProtos.RegisterRequestProto request) { + if (!isAsyncRpc) { + return registerSubordinateNamenode(unused, request); + } + asyncRouterServer(() -> server.registerSubordinateNamenode( + PBHelper.convert(request.getRegistration())), + reg -> NamenodeProtocolProtos.RegisterResponseProto.newBuilder() + .setRegistration(PBHelper.convert(reg)).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.StartCheckpointResponseProto startCheckpoint( + RpcController unused, + NamenodeProtocolProtos.StartCheckpointRequestProto request) { + if (!isAsyncRpc) { + return startCheckpoint(unused, request); + } + asyncRouterServer(() -> + server.startCheckpoint(PBHelper.convert(request.getRegistration())), + cmd -> NamenodeProtocolProtos.StartCheckpointResponseProto.newBuilder() + .setCommand(PBHelper.convert(cmd)).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.EndCheckpointResponseProto endCheckpoint( + RpcController unused, + NamenodeProtocolProtos.EndCheckpointRequestProto request) { + if (!isAsyncRpc) { + return endCheckpoint(unused, request); + } + asyncRouterServer(() -> { + server.endCheckpoint(PBHelper.convert(request.getRegistration()), + PBHelper.convert(request.getSignature())); + return null; + }, result -> VOID_END_CHECKPOINT_RESPONSE); + return null; + } + + @Override + public NamenodeProtocolProtos.GetEditLogManifestResponseProto getEditLogManifest( + RpcController unused, NamenodeProtocolProtos.GetEditLogManifestRequestProto request) { + if (!isAsyncRpc) { + return getEditLogManifest(unused, request); + } + asyncRouterServer(() -> server.getEditLogManifest(request.getSinceTxId()), + manifest -> NamenodeProtocolProtos + .GetEditLogManifestResponseProto.newBuilder() + .setManifest(PBHelper.convert(manifest)).build()); + return null; + } + + @Override + public HdfsServerProtos.VersionResponseProto versionRequest( + RpcController controller, + HdfsServerProtos.VersionRequestProto request) { + if (!isAsyncRpc) { + return versionRequest(controller, request); + } + asyncRouterServer(server::versionRequest, + info -> HdfsServerProtos.VersionResponseProto.newBuilder() + .setInfo(PBHelper.convert(info)).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.IsUpgradeFinalizedResponseProto isUpgradeFinalized( + RpcController controller, NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto request) { + if (!isAsyncRpc) { + return isUpgradeFinalized(controller, request); + } + asyncRouterServer(server::isUpgradeFinalized, + isUpgradeFinalized -> NamenodeProtocolProtos + .IsUpgradeFinalizedResponseProto.newBuilder() + .setIsUpgradeFinalized(isUpgradeFinalized).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.IsRollingUpgradeResponseProto isRollingUpgrade( + RpcController controller, NamenodeProtocolProtos.IsRollingUpgradeRequestProto request) + throws ServiceException { + if (!isAsyncRpc) { + return isRollingUpgrade(controller, request); + } + asyncRouterServer(server::isRollingUpgrade, + isRollingUpgrade -> NamenodeProtocolProtos + .IsRollingUpgradeResponseProto.newBuilder() + .setIsRollingUpgrade(isRollingUpgrade).build()); + return null; + } + + @Override + public NamenodeProtocolProtos.GetNextSPSPathResponseProto getNextSPSPath( + RpcController controller, NamenodeProtocolProtos.GetNextSPSPathRequestProto request) { + if (!isAsyncRpc) { + return getNextSPSPath(controller, request); + } + asyncRouterServer(server::getNextSPSPath, + nextSPSPath -> NamenodeProtocolProtos + .GetNextSPSPathResponseProto.newBuilder() + .setSpsPath(nextSPSPath).build()); + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolTranslatorPB.java new file mode 100644 index 0000000000000..1d56fa7b29e4a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolTranslatorPB.java @@ -0,0 +1,258 @@ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos; +import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; +import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; +import org.apache.hadoop.hdfs.server.namenode.NNStorage; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; +import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand; +import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; +import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; +import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.RpcClientUtil; +import org.apache.hadoop.util.concurrent.AsyncGet; + +import java.io.IOException; +import org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.Response; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; + +public class RouterNamenodeProtocolTranslatorPB extends NamenodeProtocolTranslatorPB{ + private final NamenodeProtocolPB rpcProxy; + + public RouterNamenodeProtocolTranslatorPB(NamenodeProtocolPB rpcProxy) { + super(rpcProxy); + this.rpcProxy = rpcProxy; + } + + @Override + public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long + minBlockSize, long timeInterval, StorageType storageType) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBlocks(datanode, size, minBlockSize, timeInterval, storageType); + } + NamenodeProtocolProtos.GetBlocksRequestProto.Builder builder = + NamenodeProtocolProtos.GetBlocksRequestProto.newBuilder() + .setDatanode(PBHelperClient.convert((DatanodeID)datanode)).setSize(size) + .setMinBlockSize(minBlockSize).setTimeInterval(timeInterval); + if (storageType != null) { + builder.setStorageType(PBHelperClient.convertStorageType(storageType)); + } + NamenodeProtocolProtos.GetBlocksRequestProto req = builder.build(); + + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getBlocks(NULL_CONTROLLER, req)); + asyncResponse(() -> PBHelper.convert( + asyncGet.get(-1, null).getBlocks())); + return null; + } + + @Override + public ExportedBlockKeys getBlockKeys() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBlockKeys(); + } + AsyncGet asyncGet = + asyncIpc(() -> + rpcProxy.getBlockKeys(NULL_CONTROLLER, VOID_GET_BLOCKKEYS_REQUEST)); + asyncResponse(() -> { + NamenodeProtocolProtos.GetBlockKeysResponseProto rsp = + asyncGet.get(-1, null); + return rsp.hasKeys() ? PBHelper.convert(rsp.getKeys()) : null; + }); + return null; + } + + @Override + public long getTransactionID() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getTransactionID(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getTransactionId(NULL_CONTROLLER, + VOID_GET_TRANSACTIONID_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null).getTxId()); + return -1; + } + + @Override + public long getMostRecentCheckpointTxId() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getMostRecentCheckpointTxId(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getMostRecentCheckpointTxId(NULL_CONTROLLER, + NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto.getDefaultInstance())); + asyncResponse((Response) () -> asyncGet.get(-1, null).getTxId()); + return -1; + } + + @Override + public long getMostRecentNameNodeFileTxId(NNStorage.NameNodeFile nnf) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getMostRecentNameNodeFileTxId(nnf); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getMostRecentNameNodeFileTxId(NULL_CONTROLLER, + NamenodeProtocolProtos.GetMostRecentNameNodeFileTxIdRequestProto.newBuilder() + .setNameNodeFile(nnf.toString()).build())); + asyncResponse(() -> asyncGet.get(-1, null).getTxId()); + return -1; + } + + @Override + public CheckpointSignature rollEditLog() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.rollEditLog(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.rollEditLog(NULL_CONTROLLER, + VOID_ROLL_EDITLOG_REQUEST)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getSignature())); + return null; + } + + @Override + public NamespaceInfo versionRequest() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.versionRequest(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.versionRequest(NULL_CONTROLLER, + VOID_VERSION_REQUEST)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getInfo())); + return null; + } + + @Override + public void errorReport(NamenodeRegistration registration, int errorCode, + String msg) throws IOException { + if (!Client.isAsynchronousMode()) { + super.errorReport(registration, errorCode, msg); + return; + } + NamenodeProtocolProtos.ErrorReportRequestProto req = NamenodeProtocolProtos.ErrorReportRequestProto.newBuilder() + .setErrorCode(errorCode).setMsg(msg) + .setRegistration(PBHelper.convert(registration)).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.errorReport(NULL_CONTROLLER, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public NamenodeRegistration registerSubordinateNamenode( + NamenodeRegistration registration) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.registerSubordinateNamenode(registration); + } + NamenodeProtocolProtos.RegisterRequestProto req = NamenodeProtocolProtos.RegisterRequestProto.newBuilder() + .setRegistration(PBHelper.convert(registration)).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.registerSubordinateNamenode(NULL_CONTROLLER, req)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getRegistration())); + return null; + } + + @Override + public NamenodeCommand startCheckpoint(NamenodeRegistration registration) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.startCheckpoint(registration); + } + NamenodeProtocolProtos.StartCheckpointRequestProto req = NamenodeProtocolProtos.StartCheckpointRequestProto.newBuilder() + .setRegistration(PBHelper.convert(registration)).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.startCheckpoint(NULL_CONTROLLER, req)); + asyncResponse(() -> { + HdfsServerProtos.NamenodeCommandProto cmd = + asyncGet.get(-1, null).getCommand(); + return PBHelper.convert(cmd); + }); + return null; + } + + @Override + public void endCheckpoint(NamenodeRegistration registration, + CheckpointSignature sig) throws IOException { + if (!Client.isAsynchronousMode()) { + super.endCheckpoint(registration, sig); + return; + } + NamenodeProtocolProtos.EndCheckpointRequestProto req = NamenodeProtocolProtos.EndCheckpointRequestProto.newBuilder() + .setRegistration(PBHelper.convert(registration)) + .setSignature(PBHelper.convert(sig)).build(); + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.endCheckpoint(NULL_CONTROLLER, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public RemoteEditLogManifest getEditLogManifest(long sinceTxId) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getEditLogManifest(sinceTxId); + } + NamenodeProtocolProtos.GetEditLogManifestRequestProto req = NamenodeProtocolProtos.GetEditLogManifestRequestProto + .newBuilder().setSinceTxId(sinceTxId).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getEditLogManifest(NULL_CONTROLLER, req)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getManifest())); + return null; + } + + @Override + public boolean isUpgradeFinalized() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.isUpgradeFinalized(); + } + NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto req = NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto + .newBuilder().build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.isUpgradeFinalized(NULL_CONTROLLER, req)); + asyncResponse(() -> asyncGet.get(-1, null).getIsUpgradeFinalized()); + return false; + } + + @Override + public boolean isRollingUpgrade() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.isRollingUpgrade(); + } + NamenodeProtocolProtos.IsRollingUpgradeRequestProto req = NamenodeProtocolProtos.IsRollingUpgradeRequestProto + .newBuilder().build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.isRollingUpgrade(NULL_CONTROLLER, req)); + asyncResponse(() -> asyncGet.get(-1, null).getIsRollingUpgrade()); + return false; + } + + @Override + public Long getNextSPSPath() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getNextSPSPath(); + } + NamenodeProtocolProtos.GetNextSPSPathRequestProto req = + NamenodeProtocolProtos.GetNextSPSPathRequestProto.newBuilder().build(); + AsyncGet ayncGet = + asyncIpc(() -> rpcProxy.getNextSPSPath(NULL_CONTROLLER, req)); + asyncResponse(() -> { + NamenodeProtocolProtos.GetNextSPSPathResponseProto nextSPSPath = + ayncGet.get(-1, null); + return nextSPSPath.hasSpsPath() ? nextSPSPath.getSpsPath() : null; + }); + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolServerSideTranslatorPB.java new file mode 100644 index 0000000000000..45d2e5578a9be --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolServerSideTranslatorPB.java @@ -0,0 +1,62 @@ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.security.RefreshUserMappingsProtocol; +import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolServerSideTranslatorPB; +import org.apache.hadoop.thirdparty.protobuf.Message; +import org.apache.hadoop.thirdparty.protobuf.RpcController; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationRequestProto; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationResponseProto; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsResponseProto; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncRouterServer; + +public class RouterRefreshUserMappingsProtocolServerSideTranslatorPB + extends RefreshUserMappingsProtocolServerSideTranslatorPB { + + private final RouterRpcServer server; + private final boolean isAsyncRpc; + + public RouterRefreshUserMappingsProtocolServerSideTranslatorPB( + RefreshUserMappingsProtocol impl) { + super(impl); + this.server = (RouterRpcServer) impl; + this.isAsyncRpc = server.isAsync(); + } + + @Override + public RefreshUserToGroupsMappingsResponseProto + refreshUserToGroupsMappings( + RpcController controller, + RefreshUserToGroupsMappingsRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.refreshUserToGroupsMappings(controller, request); + } + asyncRouterServer(() -> { + server.refreshUserToGroupsMappings(); + return null; + }, result -> + VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE); + return null; + } + + @Override + public RefreshSuperUserGroupsConfigurationResponseProto + refreshSuperUserGroupsConfiguration( + RpcController controller, + RefreshSuperUserGroupsConfigurationRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.refreshSuperUserGroupsConfiguration(controller, request); + } + asyncRouterServer(() -> { + server.refreshSuperUserGroupsConfiguration(); + return null; + }, result -> + VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE); + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolTranslatorPB.java new file mode 100644 index 0000000000000..436e4e8ef53c3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolTranslatorPB.java @@ -0,0 +1,53 @@ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos; +import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB; +import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolPB; + +import org.apache.hadoop.util.concurrent.AsyncGet; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; + + +public class RouterRefreshUserMappingsProtocolTranslatorPB + extends RefreshUserMappingsProtocolClientSideTranslatorPB { + private final RefreshUserMappingsProtocolPB rpcProxy; + public RouterRefreshUserMappingsProtocolTranslatorPB(RefreshUserMappingsProtocolPB rpcProxy) { + super(rpcProxy); + this.rpcProxy = rpcProxy; + } + + @Override + public void refreshUserToGroupsMappings() throws IOException { + if (!Client.isAsynchronousMode()) { + super.refreshUserToGroupsMappings(); + return; + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.refreshUserToGroupsMappings(NULL_CONTROLLER, + VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public void refreshSuperUserGroupsConfiguration() throws IOException { + if (!Client.isAsynchronousMode()) { + super.refreshSuperUserGroupsConfiguration(); + return; + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.refreshSuperUserGroupsConfiguration(NULL_CONTROLLER, + VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java index 5d5f9fb8aa12a..ab6a70be233bb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCMetrics.java @@ -302,6 +302,10 @@ public void addProcessingTime(long time) { processingOp.incr(); } + public void incrProcessingOp() { + processingOp.incr(); + } + @Override public double getProcessingAvg() { return processing.lastStat().mean(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java index 3a0fa2016d84e..b728545ffafb4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java @@ -151,23 +151,37 @@ public long proxyOp() { return Thread.currentThread().getId(); } + @Override + public void incrProcessingOp() { + if (metrics != null) { + metrics.incrProcessingOp(); + } + } + @Override public void proxyOpComplete(boolean success, String nsId, FederationNamenodeServiceState state) { + proxyOpComplete(success, nsId, state, getProxyTime()); + } + + @Override + public void proxyOpComplete( + boolean success, String nsId, FederationNamenodeServiceState state, long costMs) { if (success) { - long proxyTime = getProxyTime(); - if (proxyTime >= 0) { + if (costMs >= 0) { if (metrics != null && !CONCURRENT.equals(nsId)) { - metrics.addProxyTime(proxyTime, state); + metrics.addProxyTime(costMs, state); } if (nameserviceRPCMetricsMap != null && nameserviceRPCMetricsMap.containsKey(nsId)) { - nameserviceRPCMetricsMap.get(nsId).addProxyTime(proxyTime); + nameserviceRPCMetricsMap.get(nsId).addProxyTime(costMs); } } } + System.out.println("zjtest" + nsId + ":" + state); } + @Override public void proxyOpFailureStandby(String nsId) { if (metrics != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java index f90939acab558..2844dc86513fd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo; import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; import org.apache.hadoop.hdfs.server.federation.router.Router; +import org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil; import org.apache.hadoop.hdfs.server.federation.router.RouterClientProtocol; import org.apache.hadoop.hdfs.server.federation.router.SubClusterTimeoutException; import org.apache.hadoop.hdfs.server.federation.store.MembershipStore; @@ -468,6 +469,9 @@ private String getNodesImpl(final DatanodeReportType type) { this.router.getRpcServer().getClientProtocolModule(); DatanodeStorageReport[] datanodeStorageReports = clientProtocol.getDatanodeStorageReport(type, false, dnReportTimeOut); + if (router.isEnableAsync()) { + datanodeStorageReports = (DatanodeStorageReport[]) RouterAsyncRpcUtil.getResult(); + } for (DatanodeStorageReport datanodeStorageReport : datanodeStorageReports) { DatanodeInfo node = datanodeStorageReport.getDatanodeInfo(); StorageReport[] storageReports = datanodeStorageReport.getStorageReports(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java index 0a28688c916d3..228361a503700 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java @@ -60,6 +60,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; import org.apache.hadoop.hdfs.server.federation.router.Router; +import org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil; import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; import org.apache.hadoop.hdfs.server.federation.router.RouterServiceState; import org.apache.hadoop.hdfs.server.federation.router.security.RouterSecurityManager; @@ -559,7 +560,12 @@ public String getNodeUsage() { DatanodeInfo[] live = null; if (this.enableGetDNUsage) { RouterRpcServer rpcServer = this.router.getRpcServer(); - live = rpcServer.getDatanodeReport(DatanodeReportType.LIVE, false, timeOut); + if (rpcServer.isAsync()) { + rpcServer.getDatanodeReportAsync(DatanodeReportType.LIVE, false, timeOut); + live = (DatanodeInfo[]) RouterAsyncRpcUtil.getResult(); + } else { + live = rpcServer.getDatanodeReport(DatanodeReportType.LIVE, false, timeOut); + } } else { LOG.debug("Getting node usage is disabled."); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java index 933ecf070091a..549d5ee7af44b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java @@ -33,6 +33,7 @@ import java.util.TreeSet; import java.util.concurrent.ConcurrentHashMap; +import com.sun.javafx.UnmodifiableArrayList; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.server.federation.store.DisabledNameserviceStore; @@ -239,7 +240,7 @@ public List getNamenodesForNameserviceId( List ret = cacheNS.get(Pair.of(nsId, listObserversFirst)); if (ret != null) { - return shuffleObserverNN(ret, listObserversFirst); + return new ArrayList<>(shuffleObserverNN(ret, listObserversFirst)); } // Not cached, generate the value diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/AsyncErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/AsyncErasureCoding.java new file mode 100644 index 0000000000000..f0896bda6582f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/AsyncErasureCoding.java @@ -0,0 +1,123 @@ +package org.apache.hadoop.hdfs.server.federation.router; + +import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; +import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats; +import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo; +import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.asyncRequestThenApply; + +import static org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.merge; + +public class AsyncErasureCoding extends ErasureCoding{ + /** RPC server to receive client calls. */ + private final RouterRpcServer rpcServer; + /** RPC clients to connect to the Namenodes. */ + private final RouterRpcClient rpcClient; + /** Interface to identify the active NN for a nameservice or blockpool ID. */ + private final ActiveNamenodeResolver namenodeResolver; + + public AsyncErasureCoding(RouterRpcServer server) { + super(server); + this.rpcServer = server; + this.rpcClient = this.rpcServer.getRPCClient(); + this.namenodeResolver = this.rpcClient.getNamenodeResolver(); + } + + public ErasureCodingPolicyInfo[] getErasureCodingPolicies() + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getErasureCodingPolicies"); + Set nss = namenodeResolver.getNamespaces(); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent( + nss, method, true, false, + ErasureCodingPolicyInfo[].class), + ret -> merge(ret, ErasureCodingPolicyInfo.class), + ErasureCodingPolicyInfo[].class); + } + + public Map getErasureCodingCodecs() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getErasureCodingCodecs"); + Set nss = namenodeResolver.getNamespaces(); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent( + nss, method, true, false, Map.class), + retCodecs -> { + Map ret = new HashMap<>(); + Object obj = retCodecs; + @SuppressWarnings("unchecked") + Map> results = + (Map>)obj; + Collection> allCodecs = results.values(); + for (Map codecs : allCodecs) { + ret.putAll(codecs); + } + return ret; + }, Map.class); + } + + public AddErasureCodingPolicyResponse[] addErasureCodingPolicies( + ErasureCodingPolicy[] policies) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + RemoteMethod method = new RemoteMethod("addErasureCodingPolicies", + new Class[] {ErasureCodingPolicy[].class}, new Object[] {policies}); + Set nss = namenodeResolver.getNamespaces(); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent( + nss, method, true, false, AddErasureCodingPolicyResponse[].class), + ret -> merge(ret, AddErasureCodingPolicyResponse.class), + AddErasureCodingPolicyResponse[].class); + } + + public ECTopologyVerifierResult getECTopologyResultForPolicies( + String[] policyNames) throws IOException { + RemoteMethod method = new RemoteMethod("getECTopologyResultForPolicies", + new Class[] {String[].class}, new Object[] {policyNames}); + Set nss = namenodeResolver.getNamespaces(); + if (nss.isEmpty()) { + throw new IOException("No namespace availaible."); + } + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent( + nss, method, true, false, + ECTopologyVerifierResult.class), + ret -> { + for (Map.Entry entry : ret + .entrySet()) { + if (!entry.getValue().isSupported()) { + return entry.getValue(); + } + } + // If no negative result, return the result from the first namespace. + return ret.get(nss.iterator().next()); + }, ECTopologyVerifierResult.class); + } + + public ECBlockGroupStats getECBlockGroupStats() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getECBlockGroupStats"); + Set nss = namenodeResolver.getNamespaces(); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent( + nss, method, true, false, + ECBlockGroupStats.class), + allStats -> ECBlockGroupStats.merge(allStats.values()), + ECBlockGroupStats.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/AsyncQuota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/AsyncQuota.java new file mode 100644 index 0000000000000..4c0a5a9a0d309 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/AsyncQuota.java @@ -0,0 +1,37 @@ +package org.apache.hadoop.hdfs.server.federation.router; + +import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +public class AsyncQuota extends Quota{ + public AsyncQuota(Router router, RouterRpcServer server) { + super(router, server); + } + + /** + * Get aggregated quota usage for the federation path. + * @param path Federation path. + * @return Aggregated quota. + * @throws IOException If the quota system is disabled. + */ + public QuotaUsage getQuotaUsage(String path) throws IOException { + getEachQuotaUsage(path); + CompletableFuture completableFuture = + RouterAsyncRpcUtil.getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results = (Map) o; + try { + return AsyncQuota.super.aggregateQuota(path, results); + } catch (IOException e) { + throw new CompletionException(e); + } + }); + RouterAsyncRpcUtil.setCurCompletableFuture(completableFuture); + return RouterAsyncRpcUtil.asyncReturn(QuotaUsage.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java index c13debf571c7d..5c3f95b2c8286 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java @@ -32,6 +32,10 @@ import javax.net.SocketFactory; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.hdfs.protocolPB.RouterClientProtocolTranslatorPB; +import org.apache.hadoop.hdfs.protocolPB.RouterGetUserMappingsProtocolTranslatorPB; +import org.apache.hadoop.hdfs.protocolPB.RouterNamenodeProtocolTranslatorPB; +import org.apache.hadoop.hdfs.protocolPB.RouterRefreshUserMappingsProtocolTranslatorPB; import org.apache.hadoop.ipc.AlignmentContext; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -41,9 +45,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB; -import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB; -import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolTranslatorPB; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.retry.RetryPolicy; @@ -55,10 +57,8 @@ import org.apache.hadoop.security.SaslRpcServer; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB; import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolPB; import org.apache.hadoop.tools.GetUserMappingsProtocol; -import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB; import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB; import org.apache.hadoop.util.Time; import org.eclipse.jetty.util.ajax.JSON; @@ -117,15 +117,15 @@ public class ConnectionPool { static { PROTO_MAP.put(ClientProtocol.class, new ProtoImpl(ClientNamenodeProtocolPB.class, - ClientNamenodeProtocolTranslatorPB.class)); + RouterClientProtocolTranslatorPB.class)); PROTO_MAP.put(NamenodeProtocol.class, new ProtoImpl( - NamenodeProtocolPB.class, NamenodeProtocolTranslatorPB.class)); + NamenodeProtocolPB.class, RouterNamenodeProtocolTranslatorPB.class)); PROTO_MAP.put(RefreshUserMappingsProtocol.class, new ProtoImpl(RefreshUserMappingsProtocolPB.class, - RefreshUserMappingsProtocolClientSideTranslatorPB.class)); + RouterRefreshUserMappingsProtocolTranslatorPB.class)); PROTO_MAP.put(GetUserMappingsProtocol.class, new ProtoImpl(GetUserMappingsProtocolPB.class, - GetUserMappingsProtocolClientSideTranslatorPB.class)); + RouterGetUserMappingsProtocolTranslatorPB.class)); } /** Class to store the protocol implementation. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java index e2ff4728cf2aa..75aaabb3b08b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ErasureCoding.java @@ -144,9 +144,8 @@ public ErasureCodingPolicy getErasureCodingPolicy(String src) rpcServer.getLocationsForPath(src, false, false); RemoteMethod remoteMethod = new RemoteMethod("getErasureCodingPolicy", new Class[] {String.class}, new RemoteParam()); - ErasureCodingPolicy ret = rpcClient.invokeSequential( + return rpcClient.invokeSequential( locations, remoteMethod, null, null); - return ret; } public void setErasureCodingPolicy(String src, String ecPolicyName) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java index e19e51b5733ac..859f7a8548153 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java @@ -149,10 +149,9 @@ Map getEachQuotaUsage(String path) final List quotaLocs = getValidQuotaLocations(path); RemoteMethod method = new RemoteMethod("getQuotaUsage", new Class[] {String.class}, new RemoteParam()); - Map results = rpcClient.invokeConcurrent( - quotaLocs, method, true, false, QuotaUsage.class); - return results; + return rpcClient.invokeConcurrent( + quotaLocs, method, true, false, QuotaUsage.class); } /** @@ -361,7 +360,7 @@ public static boolean andByStorageType(Predicate predicate) { * @return List of quota remote locations. * @throws IOException */ - private List getQuotaRemoteLocations(String path) + protected List getQuotaRemoteLocations(String path) throws IOException { List locations = new ArrayList<>(); RouterQuotaManager manager = this.router.getQuotaManager(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java index 3d996b3e849f8..a112756e56dbc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java @@ -142,6 +142,7 @@ public class Router extends CompositeService implements /** State of the Router. */ private RouterServiceState state = RouterServiceState.UNINITIALIZED; + private boolean enableAsync; ///////////////////////////////////////////////////////// @@ -876,4 +877,7 @@ public void setConf(Configuration conf) { this.conf = conf; } + public boolean isEnableAsync() { + return true; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java index 2d96ab1be359b..77dda65b8473f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java @@ -30,6 +30,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntriesRequest; import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntriesResponse; @@ -105,6 +107,10 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.thirdparty.protobuf.BlockingService; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.getCompletableFuture; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.getResult; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.setCurCompletableFuture; + /** * This class is responsible for handling all the Admin calls to the HDFS * router. It is created, started, and stopped by {@link Router}. @@ -617,6 +623,9 @@ public GetDestinationResponse getDestination( private List getDestinationNameServices( GetDestinationRequest request, List locations) throws IOException { + if (router.isEnableAsync()) { + return getDestinationNameServicesAsync(request, locations); + } final String src = request.getSrcPath(); final List nsIds = new ArrayList<>(); RouterRpcServer rpcServer = this.router.getRpcServer(); @@ -639,6 +648,37 @@ private List getDestinationNameServices( return nsIds; } + private List getDestinationNameServicesAsync( + GetDestinationRequest request, List locations) + throws IOException { + final String src = request.getSrcPath(); + RouterRpcServer rpcServer = this.router.getRpcServer(); + RouterRpcClient rpcClient = rpcServer.getRPCClient(); + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + try { + rpcClient.invokeConcurrent( + locations, method, false, false, HdfsFileStatus.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + final List nsIds = new ArrayList<>(); + Map responses = + (Map) o; + for (RemoteLocation location : locations) { + if (responses.get(location) != null) { + nsIds.add(location.getNameserviceId()); + } + } + return nsIds; + }); + setCurCompletableFuture(completableFuture); + } catch (IOException ioe) { + LOG.error("Cannot get location for {}: {}", + src, ioe.getMessage()); + } + return (List) getResult(); + } + /** * Verify the file exists in destination nameservices to avoid dangling * mount points. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncCacheAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncCacheAdmin.java new file mode 100644 index 0000000000000..b8a7e03ea735f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncCacheAdmin.java @@ -0,0 +1,91 @@ +package org.apache.hadoop.hdfs.server.federation.router; + +import org.apache.hadoop.fs.BatchedRemoteIterator; +import org.apache.hadoop.fs.CacheFlag; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; +import org.apache.hadoop.hdfs.protocol.CachePoolEntry; +import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.asyncRequestThenApply; + +public class RouterAsyncCacheAdmin extends RouterCacheAdmin{ + /** RPC server to receive client calls. */ + private final RouterRpcServer rpcServer; + /** RPC clients to connect to the Namenodes. */ + private final RouterRpcClient rpcClient; + /** Interface to identify the active NN for a nameservice or blockpool ID. */ + private final ActiveNamenodeResolver namenodeResolver; + public RouterAsyncCacheAdmin(RouterRpcServer server) { + super(server); + this.rpcServer = server; + this.rpcClient = this.rpcServer.getRPCClient(); + this.namenodeResolver = this.rpcClient.getNamenodeResolver(); + } + + public long addCacheDirective( + CacheDirectiveInfo path, EnumSet flags) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.WRITE, true); + final List locations = + rpcServer.getLocationsForPath(path.getPath().toString(), true, + false); + RemoteMethod method = new RemoteMethod("addCacheDirective", + new Class[] {CacheDirectiveInfo.class, EnumSet.class}, + new RemoteParam(getRemoteMap(path, locations)), flags); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent( + locations, method, false, false, long.class), + response -> response.values().iterator().next(), + Long.class); + } + + public BatchedRemoteIterator.BatchedEntries listCacheDirectives( + long prevId, + CacheDirectiveInfo filter) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ, true); + CompletableFuture completableFuture = null; + if (filter.getPath() != null) { + final List locations = rpcServer + .getLocationsForPath(filter.getPath().toString(), true, false); + RemoteMethod method = new RemoteMethod("listCacheDirectives", + new Class[] {long.class, CacheDirectiveInfo.class}, prevId, + new RemoteParam(getRemoteMap(filter, locations))); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent(locations, method, false, + false, BatchedRemoteIterator.BatchedEntries.class), + response -> response.values().iterator().next(), + BatchedRemoteIterator.BatchedEntries.class); + } + RemoteMethod method = new RemoteMethod("listCacheDirectives", + new Class[] {long.class, CacheDirectiveInfo.class}, prevId, + filter); + Set nss = namenodeResolver.getNamespaces(); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent(nss, method, true, + false, BatchedRemoteIterator.BatchedEntries.class), + results -> results.values().iterator().next(), + BatchedRemoteIterator.BatchedEntries.class); + } + + public BatchedRemoteIterator.BatchedEntries listCachePools(String prevKey) + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ, true); + RemoteMethod method = new RemoteMethod("listCachePools", + new Class[] {String.class}, prevKey); + Set nss = namenodeResolver.getNamespaces(); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent(nss, method, true, + false, BatchedRemoteIterator.BatchedEntries.class), + results -> results.values().iterator().next(), + BatchedRemoteIterator.BatchedEntries.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncClientProtocol.java new file mode 100644 index 0000000000000..2be85076333fe --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncClientProtocol.java @@ -0,0 +1,1559 @@ +/** + * 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.server.federation.router; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.crypto.CryptoProtocolVersion; +import org.apache.hadoop.fs.BatchedRemoteIterator; +import org.apache.hadoop.fs.CacheFlag; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FsServerDefaults; +import org.apache.hadoop.fs.Options; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.fs.XAttr; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; +import org.apache.hadoop.hdfs.protocol.CachePoolEntry; +import org.apache.hadoop.hdfs.protocol.CachePoolInfo; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats; +import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult; +import org.apache.hadoop.hdfs.protocol.EncryptionZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats; +import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing; +import org.apache.hadoop.hdfs.protocol.SnapshotStatus; +import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; +import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; +import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo; +import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.federation.resolver.RouterResolveException; +import org.apache.hadoop.hdfs.server.federation.store.records.MountTable; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; +import org.apache.hadoop.io.EnumSetWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.EnumSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CompletionStage; +import java.util.function.Function; + +import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.updateMountPointStatus; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.getCompletableFuture; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.setCurCompletableFuture; + +public class RouterAsyncClientProtocol extends RouterClientProtocol { + private static final Logger LOG = + LoggerFactory.getLogger(RouterAsyncClientProtocol.class.getName()); + + private final RouterSnapshot asyncSnapshotProto; + private final AsyncErasureCoding asyncErasureCoding; + private final RouterAsyncCacheAdmin routerAsyncCacheAdmin; + private volatile FsServerDefaults serverDefaults; + private final RouterAsyncStoragePolicy asyncstoragePolicy; + + RouterAsyncClientProtocol(Configuration conf, RouterRpcServer rpcServer) { + super(conf, rpcServer); + asyncSnapshotProto = new RouterAsyncSnapshot(rpcServer); + asyncErasureCoding = new AsyncErasureCoding(rpcServer); + routerAsyncCacheAdmin = new RouterAsyncCacheAdmin(rpcServer); + asyncstoragePolicy = new RouterAsyncStoragePolicy(rpcServer); + } + + @Override + public FsServerDefaults getServerDefaults() throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + long serverDefaultsLastUpdate = getServerDefaultsLastUpdate(); + long serverDefaultsValidityPeriod = getServerDefaultsValidityPeriod(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + long now = Time.monotonicNow(); + CompletableFuture completableFuture = null; + if ((serverDefaults == null) || (now - serverDefaultsLastUpdate + > serverDefaultsValidityPeriod)) { + RemoteMethod method = new RemoteMethod("getServerDefaults"); + serverDefaults = + rpcServer.invokeAtAvailableNsAsync(method, FsServerDefaults.class); + completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + serverDefaults = (FsServerDefaults) o; + RouterAsyncClientProtocol.super.setServerDefaultsLastUpdate(now); + return o; + }); + } else { + completableFuture = + CompletableFuture.completedFuture(serverDefaults); + } + setCurCompletableFuture(completableFuture); + return asyncReturn(FsServerDefaults.class); + } + + @Override + public HdfsFileStatus create( + String src, FsPermission masked, String clientName, + EnumSetWritable flag, boolean createParent, short replication, + long blockSize, CryptoProtocolVersion[] supportedVersions, + String ecPolicyName, String storagePolicy) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + CompletableFuture completableFuture = null; + final Throwable[] throwable = new Throwable[1]; + if (createParent && rpcServer.isPathAll(src)) { + int index = src.lastIndexOf(Path.SEPARATOR); + String parent = src.substring(0, index); + LOG.debug("Creating {} requires creating parent {}", src, parent); + FsPermission parentPermissions = getParentPermission(masked); + mkdirs(parent, parentPermissions, createParent); + completableFuture = getCompletableFuture(); + completableFuture = completableFuture.exceptionally(e -> { + throwable[0] = e.getCause(); + throw new CompletionException(e.getCause()); + }); + } + + if (completableFuture == null) { + completableFuture = CompletableFuture.completedFuture(false); + } + RemoteMethod method = new RemoteMethod("create", + new Class[] {String.class, FsPermission.class, String.class, + EnumSetWritable.class, boolean.class, short.class, + long.class, CryptoProtocolVersion[].class, + String.class, String.class}, + new RemoteParam(), masked, clientName, flag, createParent, + replication, blockSize, supportedVersions, ecPolicyName, storagePolicy); + final RemoteLocation[] createLocation = new RemoteLocation[1]; + final List locations = + rpcServer.getLocationsForPath(src, true); + completableFuture = completableFuture.thenCompose(o -> { + try { + rpcServer.getCreateLocationAsync(src, locations); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + }).thenCompose((Function>) o -> { + createLocation[0] = (RemoteLocation) o; + try { + rpcClient.invokeSingle(createLocation[0], method, + HdfsFileStatus.class); + return getCompletableFuture().thenApply(o1 -> { + HdfsFileStatus status = (HdfsFileStatus) o1; + status.setNamespace(createLocation[0].getNameserviceId()); + return status; + }); + } catch (IOException e) { + throw new CompletionException(e); + } + }).exceptionally(Throwable::getCause).thenCompose(o -> { + if (throwable[0] != null) { + throw new CompletionException(throwable[0]); + } + if (o instanceof Throwable) { + if (o instanceof IOException) { + try { + final List newLocations = checkFaultTolerantRetry( + method, src, (IOException) o, createLocation[0], locations); + rpcClient.invokeSequential( + newLocations, method, HdfsFileStatus.class, null); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + } + throw new CompletionException((Throwable) o); + } + return CompletableFuture.completedFuture(o); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(HdfsFileStatus.class); + } + + @Override + public LastBlockWithStatus append( + String src, String clientName, + EnumSetWritable flag) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + List locations = rpcServer.getLocationsForPath(src, true); + RemoteMethod method = new RemoteMethod("append", + new Class[] {String.class, String.class, EnumSetWritable.class}, + new RemoteParam(), clientName, flag); + rpcClient.invokeSequential(method, locations, LastBlockWithStatus.class, null); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + RemoteResult result = (RemoteResult) o; + LastBlockWithStatus lbws = (LastBlockWithStatus) result.getResult(); + lbws.getFileStatus().setNamespace(result.getLocation().getNameserviceId()); + return lbws; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(LastBlockWithStatus.class); + } + + @Deprecated + @Override + public boolean rename(final String src, final String dst) + throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + RouterFederationRename rbfRename = getRbfRename(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + final List srcLocations = + rpcServer.getLocationsForPath(src, true, false); + final List dstLocations = + rpcServer.getLocationsForPath(dst, false, false); + // srcLocations may be trimmed by getRenameDestinations() + final List locs = new LinkedList<>(srcLocations); + RemoteParam dstParam = getRenameDestinations(locs, dstLocations); + if (locs.isEmpty()) { + return rbfRename.routerFedRename(src, dst, srcLocations, dstLocations); + } + RemoteMethod method = new RemoteMethod("rename", + new Class[] {String.class, String.class}, + new RemoteParam(), dstParam); + isMultiDestDirectory(src); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenCompose(o -> { + Boolean isMultiDest = (Boolean) o; + if (isMultiDest) { + if (locs.size() != srcLocations.size()) { + IOException ioe = new IOException("Rename of " + src + " to " + dst + " is not" + + " allowed. The number of remote locations for both source and" + + " target should be same."); + throw new CompletionException(ioe); + } + try { + rpcClient.invokeAll(locs, method); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + } else { + try { + rpcClient.invokeSequential(locs, method, Boolean.class, + Boolean.TRUE); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + } + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Boolean.class); + } + + @Override + public void rename2(final String src, final String dst, + final Options.Rename... options) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + RouterFederationRename rbfRename = getRbfRename(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + final List srcLocations = + rpcServer.getLocationsForPath(src, true, false); + final List dstLocations = + rpcServer.getLocationsForPath(dst, false, false); + // srcLocations may be trimmed by getRenameDestinations() + final List locs = new LinkedList<>(srcLocations); + RemoteParam dstParam = getRenameDestinations(locs, dstLocations); + if (locs.isEmpty()) { + rbfRename.routerFedRename(src, dst, srcLocations, dstLocations); + return; + } + RemoteMethod method = new RemoteMethod("rename2", + new Class[] {String.class, String.class, options.getClass()}, + new RemoteParam(), dstParam, options); + isMultiDestDirectory(src); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenCompose(o -> { + Boolean isMultiDest = (Boolean) o; + if (isMultiDest) { + if (locs.size() != srcLocations.size()) { + IOException ioe = new IOException("Rename of " + src + " to " + dst + " is not" + + " allowed. The number of remote locations for both source and" + + " target should be same."); + throw new CompletionException(ioe); + } + try { + rpcClient.invokeConcurrent(locs, method); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + } else { + try { + rpcClient.invokeSequential(locs, method, null, null); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + } + }); + setCurCompletableFuture(completableFuture); + } + + @Override + public void concat(String trg, String[] src) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + // See if the src and target files are all in the same namespace + getBlockLocations(trg, 0, 1); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenCompose(o -> { + LocatedBlocks targetBlocks = (LocatedBlocks) o; + if (targetBlocks == null) { + throw new CompletionException( + new IOException("Cannot locate blocks for target file - " + trg)); + } + LocatedBlock lastLocatedBlock = targetBlocks.getLastLocatedBlock(); + String targetBlockPoolId = lastLocatedBlock.getBlock().getBlockPoolId(); + CompletableFuture future = CompletableFuture.completedFuture(null); + for (String source : src) { + future = future.thenCompose(o1 -> { + try { + getBlockLocations(source, 0, 1); + CompletableFuture completableFuture1 = getCompletableFuture(); + return completableFuture1.thenApply(res -> { + if (res == null) { + throw new CompletionException(new IOException( + "Cannot located blocks for source file " + source)); + } + LocatedBlocks sourceBlocks1 = (LocatedBlocks) res; + String sourceBlockPoolId = + sourceBlocks1.getLastLocatedBlock().getBlock().getBlockPoolId(); + if (!sourceBlockPoolId.equals(targetBlockPoolId)) { + throw new RuntimeException( + new IOException("Cannot concatenate source file " + source + + " because it is located in a different namespace" + + " with block pool id " + sourceBlockPoolId + + " from the target file with block pool id " + + targetBlockPoolId)); + } + return res; + }); + } catch (IOException e) { + throw new CompletionException(e); + } + }); + } + return future.thenApply(r -> targetBlockPoolId); + }); + + RouterRpcClient rpcClient = getRpcClient(); + completableFuture = completableFuture.thenCompose(o -> { + String targetBlockPoolId = (String) o; + // Find locations in the matching namespace. + try { + final RemoteLocation targetDestination + = rpcServer.getLocationForPath(trg, true, targetBlockPoolId); + String[] sourceDestinations = new String[src.length]; + for (int i = 0; i < src.length; i++) { + String sourceFile = src[i]; + RemoteLocation location = + rpcServer.getLocationForPath(sourceFile, true, targetBlockPoolId); + sourceDestinations[i] = location.getDest(); + } + // Invoke + RemoteMethod method = new RemoteMethod("concat", + new Class[] {String.class, String[].class}, + targetDestination.getDest(), sourceDestinations); + rpcClient.invokeSingle(targetDestination, method, Void.class); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + }); + + setCurCompletableFuture(completableFuture); + asyncReturn(Void.class); + } + + @Override + public boolean mkdirs(String src, FsPermission masked, boolean createParent) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + final List locations = + rpcServer.getLocationsForPath(src, false); + RemoteMethod method = new RemoteMethod("mkdirs", + new Class[] {String.class, FsPermission.class, boolean.class}, + new RemoteParam(), masked, createParent); + + // Create in all locations + if (rpcServer.isPathAll(src)) { + return rpcClient.invokeAll(locations, method); + } + + CompletableFuture completableFuture + = CompletableFuture.completedFuture(false); + if (locations.size() > 1) { + // Check if this directory already exists + try { + getFileInfo(src); + completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + HdfsFileStatus fileStatus = (HdfsFileStatus) o; + if (fileStatus != null) { + // When existing, the NN doesn't return an exception; return true + return true; + } + return false; + }); + } catch (IOException ioe) { + // Can't query if this file exists or not. + LOG.error("Error getting file info for {} while proxying mkdirs: {}", + src, ioe.getMessage()); + } + } + + completableFuture = completableFuture.thenCompose(o -> { + boolean success = (boolean) o; + if (success) { + return CompletableFuture.completedFuture(true); + } + final RemoteLocation firstLocation = locations.get(0); + try { + rpcClient.invokeSingle(firstLocation, method, Boolean.class); + return getCompletableFuture(); + } catch (IOException ioe) { + throw new CompletionException(ioe); + } + }).exceptionally(e -> { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + return cause; + } else { + throw new CompletionException(cause); + } + }).thenCompose(o -> { + if (o instanceof Boolean) { + return CompletableFuture.completedFuture(o); + } + IOException ioe = (IOException) o; + final RemoteLocation firstLocation = locations.get(0); + final List newLocations; + try { + newLocations = checkFaultTolerantRetry( + method, src, ioe, firstLocation, locations); + rpcClient.invokeSequential( + newLocations, method, Boolean.class, Boolean.TRUE); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Boolean.class); + } + + @Override + public DirectoryListing getListing( + String src, byte[] startAfter, boolean needLocation) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + getListingInt(src, startAfter, needLocation); + CompletableFuture completableFuture = getCompletableFuture(); + + RouterClientProtocol.GetListingComparator comparator = getComparator(); + TreeMap nnListing = new TreeMap<>(comparator); + + completableFuture = completableFuture.thenApply(new Function() { + @Override + public Object apply(Object o) { + List> listings = + (List>) o; + int totalRemainingEntries = 0; + int remainingEntries = 0; + // Check the subcluster listing with the smallest name to make sure + // no file is skipped across subclusters + byte[] lastName = null; + boolean namenodeListingExists = false; + if (listings != null) { + for (RemoteResult result : listings) { + if (result.hasException()) { + IOException ioe = result.getException(); + if (ioe instanceof FileNotFoundException) { + RemoteLocation location = result.getLocation(); + LOG.debug("Cannot get listing from {}", location); + } else if (!isAllowPartialList()) { + throw new CompletionException(ioe); + } + } else if (result.getResult() != null) { + DirectoryListing listing = result.getResult(); + totalRemainingEntries += listing.getRemainingEntries(); + HdfsFileStatus[] partialListing = listing.getPartialListing(); + int length = partialListing.length; + if (length > 0) { + HdfsFileStatus lastLocalEntry = partialListing[length-1]; + byte[] lastLocalName = lastLocalEntry.getLocalNameInBytes(); + if (lastName == null || + comparator.compare(lastName, lastLocalName) > 0) { + lastName = lastLocalName; + } + } + } + } + + // Add existing entries + for (RemoteResult result : listings) { + DirectoryListing listing = result.getResult(); + if (listing != null) { + namenodeListingExists = true; + for (HdfsFileStatus file : listing.getPartialListing()) { + byte[] filename = file.getLocalNameInBytes(); + if (totalRemainingEntries > 0 && + comparator.compare(filename, lastName) > 0) { + // Discarding entries further than the lastName + remainingEntries++; + } else { + nnListing.put(filename, file); + } + } + remainingEntries += listing.getRemainingEntries(); + } + } + } + return new Object[] {remainingEntries, namenodeListingExists, lastName}; + } + }); + + FileSubclusterResolver subclusterResolver = getSubclusterResolver(); + // Add mount points at this level in the tree + final List children = subclusterResolver.getMountPoints(src); + if (children != null) { + // Get the dates for each mount point + Map dates = getMountPointDates(src); + + // Create virtual folder with the mount name + for (String child : children) { + completableFuture = completableFuture.thenCompose(o -> { + Object[] args = (Object[]) o; + int remainingEntries = (int) args[0]; + byte[] lastName = (byte[]) args[2]; + long date = 0; + if (dates != null && dates.containsKey(child)) { + date = dates.get(child); + } + Path childPath = new Path(src, child); + getMountPointStatus(childPath.toString(), 0, date); + CompletableFuture future = getCompletableFuture(); + future = future.thenApply(o1 -> { + HdfsFileStatus dirStatus = (HdfsFileStatus) o1; + // if there is no subcluster path, always add mount point + byte[] bChild = DFSUtil.string2Bytes(child); + if (lastName == null) { + nnListing.put(bChild, dirStatus); + } else { + if (shouldAddMountPoint(bChild, + lastName, startAfter, remainingEntries)) { + // This may overwrite existing listing entries with the mount point + // TODO don't add if already there? + nnListing.put(bChild, dirStatus); + } + } + return new Object[] {remainingEntries, args[1], lastName}; + }); + return future; + }); + } + + completableFuture = completableFuture.thenApply(o -> { + Object[] args = (Object[]) o; + int remainingEntries = (int) args[0]; + // Update the remaining count to include left mount points + if (nnListing.size() > 0) { + byte[] lastListing = nnListing.lastKey(); + for (int i = 0; i < children.size(); i++) { + byte[] bChild = DFSUtil.string2Bytes(children.get(i)); + if (comparator.compare(bChild, lastListing) > 0) { + remainingEntries += (children.size() - i); + break; + } + } + } + return new Object[] {remainingEntries, args[1], args[2]}; + }); + + } + + completableFuture = completableFuture.thenApply(o -> { + Object[] args = (Object[]) o; + int remainingEntries = (int) args[0]; + boolean namenodeListingExists = (boolean) args[1]; + if (!namenodeListingExists && nnListing.size() == 0 && children == null) { + // NN returns a null object if the directory cannot be found and has no + // listing. If we didn't retrieve any NN listing data, and there are no + // mount points here, return null. + return null; + } + + // Generate combined listing + HdfsFileStatus[] combinedData = new HdfsFileStatus[nnListing.size()]; + combinedData = nnListing.values().toArray(combinedData); + return new DirectoryListing(combinedData, remainingEntries); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(DirectoryListing.class); + } + + /** + * Get listing on remote locations. + */ + @Override + List> getListingInt( + String src, byte[] startAfter, boolean needLocation) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + try { + List locations = + rpcServer.getLocationsForPath(src, false, false); + // Locate the dir and fetch the listing. + if (locations.isEmpty()) { + setCurCompletableFuture(CompletableFuture.completedFuture(new ArrayList<>())); + return asyncReturn(List.class); + } + RemoteMethod method = new RemoteMethod("getListing", + new Class[] {String.class, startAfter.getClass(), boolean.class}, + new RemoteParam(), startAfter, needLocation); + rpcClient.invokeConcurrent(locations, method, false, -1, + DirectoryListing.class); + } catch (NoLocationException | RouterResolveException e) { + LOG.debug("Cannot get locations for {}, {}.", src, e.getMessage()); + setCurCompletableFuture(CompletableFuture.completedFuture(new ArrayList<>())); + } + return asyncReturn(List.class); + } + + + HdfsFileStatus getMountPointStatus( + String name, int childrenNum, long date, boolean setPath) { + FileSubclusterResolver subclusterResolver = getSubclusterResolver(); + long modTime = date; + long accessTime = date; + FsPermission permission = FsPermission.getDirDefault(); + String owner = getSuperUser(); + String group = getSuperGroup(); + EnumSet flags = + EnumSet.noneOf(HdfsFileStatus.Flags.class); + CompletableFuture completableFuture = null; + if (subclusterResolver instanceof MountTableResolver) { + try { + String mName = name.startsWith("/") ? name : "/" + name; + MountTableResolver mountTable = (MountTableResolver) subclusterResolver; + MountTable entry = mountTable.getMountPoint(mName); + if (entry != null) { + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + getFileInfoAll(entry.getDestinations(), method, getMountStatusTimeOut()); + completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + HdfsFileStatus fInfo = (HdfsFileStatus) o; + if (fInfo != null) { + return new Object[] { + fInfo.getPermission(), fInfo.getOwner(), fInfo.getGroup(), + fInfo.getChildrenNum(), DFSUtil + .getFlags(fInfo.isEncrypted(), fInfo.isErasureCoded(), + fInfo.isSnapshotEnabled(), fInfo.hasAcl())}; + } + return new Object[] {entry.getMode(), entry.getOwnerName(), entry.getGroupName(), + childrenNum, flags}; + }); + } + } catch (IOException e) { + LOG.error("Cannot get mount point: {}", e.getMessage()); + completableFuture = CompletableFuture.completedFuture( + new Object[]{permission, owner, group, childrenNum, flags}); + } + } else { + try { + UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + owner = ugi.getUserName(); + group = ugi.getPrimaryGroupName(); + completableFuture = + CompletableFuture.completedFuture(new Object[]{permission, owner, group, + childrenNum, flags}); + } catch (IOException e) { + String msg = "Cannot get remote user: " + e.getMessage(); + if (UserGroupInformation.isSecurityEnabled()) { + LOG.error(msg); + } else { + LOG.debug(msg); + } + completableFuture = CompletableFuture.completedFuture( + new Object[]{permission, owner, group, childrenNum, flags}); + } + } + + completableFuture = completableFuture.thenApply(o -> { + Object[] args = (Object[]) o; + long inodeId = 0; + HdfsFileStatus.Builder builder = new HdfsFileStatus.Builder(); + if (setPath) { + Path path = new Path(name); + String nameStr = path.getName(); + builder.path(DFSUtil.string2Bytes(nameStr)); + } + + return builder.isdir(true) + .mtime(modTime) + .atime(accessTime) + .perm((FsPermission) args[0]) + .owner((String) args[1]) + .group((String) args[2]) + .symlink(new byte[0]) + .fileId(inodeId) + .children((Integer) args[3]) + .flags((EnumSet) args[4]) + .build(); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(HdfsFileStatus.class); + } + + + private HdfsFileStatus getFileInfoAll( + final List locations, + final RemoteMethod method, long timeOutMs) throws IOException { + + RouterRpcClient rpcClient = getRpcClient(); + // Get the file info from everybody + rpcClient.invokeConcurrent(locations, method, false, false, timeOutMs, + HdfsFileStatus.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results = (Map) o; + int children = 0; + // We return the first file + HdfsFileStatus dirStatus = null; + for (RemoteLocation loc : locations) { + HdfsFileStatus fileStatus = results.get(loc); + if (fileStatus != null) { + children += fileStatus.getChildrenNum(); + if (!fileStatus.isDirectory()) { + return fileStatus; + } else if (dirStatus == null) { + dirStatus = fileStatus; + } + } + } + if (dirStatus != null) { + return updateMountPointStatus(dirStatus, children); + } + return null; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(HdfsFileStatus.class); + } + + @Override + public boolean recoverLease(String src, String clientName) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + final List locations = + rpcServer.getLocationsForPath(src, true, false); + RemoteMethod method = new RemoteMethod("recoverLease", + new Class[] {String.class, String.class}, new RemoteParam(), + clientName); + rpcClient.invokeSequential(locations, method, Boolean.class, null); + return asyncReturn(Boolean.class); + } + + @Override + public long[] getStats() throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + RemoteMethod method = new RemoteMethod("getStats"); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, false, long[].class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results + = (Map) o; + long[] combinedData = new long[STATS_ARRAY_LENGTH]; + for (long[] data : results.values()) { + for (int i = 0; i < combinedData.length && i < data.length; i++) { + if (data[i] >= 0) { + combinedData[i] += data[i]; + } + } + } + return combinedData; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(long[].class); + } + + @Override + public ErasureCodingPolicyInfo[] getErasureCodingPolicies() + throws IOException { + return asyncErasureCoding.getErasureCodingPolicies(); + } + + @Override + public Map getErasureCodingCodecs() throws IOException { + return asyncErasureCoding.getErasureCodingCodecs(); + } + + @Override + public AddErasureCodingPolicyResponse[] addErasureCodingPolicies( + ErasureCodingPolicy[] policies) throws IOException { + return asyncErasureCoding.addErasureCodingPolicies(policies); + } + + @Override + public void removeErasureCodingPolicy(String ecPolicyName) + throws IOException { + asyncErasureCoding.removeErasureCodingPolicy(ecPolicyName); + } + + @Override + public void disableErasureCodingPolicy(String ecPolicyName) + throws IOException { + asyncErasureCoding.disableErasureCodingPolicy(ecPolicyName); + } + + @Override + public void enableErasureCodingPolicy(String ecPolicyName) + throws IOException { + asyncErasureCoding.enableErasureCodingPolicy(ecPolicyName); + } + + @Override + public ErasureCodingPolicy getErasureCodingPolicy(String src) + throws IOException { + return asyncErasureCoding.getErasureCodingPolicy(src); + } + + @Override + public void setErasureCodingPolicy(String src, String ecPolicyName) + throws IOException { + asyncErasureCoding.setErasureCodingPolicy(src, ecPolicyName); + } + + @Override + public void unsetErasureCodingPolicy(String src) throws IOException { + asyncErasureCoding.unsetErasureCodingPolicy(src); + } + + @Override + public ECTopologyVerifierResult getECTopologyResultForPolicies( + String... policyNames) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED, true); + return asyncErasureCoding.getECTopologyResultForPolicies(policyNames); + } + + @Override + public ECBlockGroupStats getECBlockGroupStats() throws IOException { + return asyncErasureCoding.getECBlockGroupStats(); + } + + + @Override + public ReplicatedBlockStats getReplicatedBlockStats() throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getReplicatedBlockStats"); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, + false, ReplicatedBlockStats.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map ret = + (Map) o; + return ReplicatedBlockStats.merge(ret.values()); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(ReplicatedBlockStats.class); + } + + @Override + public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type) + throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + return rpcServer.getDatanodeReportAsync(type, true, 0); + } + + @Override + public DatanodeStorageReport[] getDatanodeStorageReport( + HdfsConstants.DatanodeReportType type) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + rpcServer.getDatanodeStorageReportMapAsync(type); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map dnSubcluster = + (Map) o; + return mergeDtanodeStorageReport(dnSubcluster); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(DatanodeStorageReport[].class); + } + + public DatanodeStorageReport[] getDatanodeStorageReport( + HdfsConstants.DatanodeReportType type, boolean requireResponse, long timeOutMs) + throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + rpcServer.getDatanodeStorageReportMapAsync(type, requireResponse, timeOutMs); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map dnSubcluster = + (Map) o; + return mergeDtanodeStorageReport(dnSubcluster); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(DatanodeStorageReport[].class); + } + + @Override + public boolean setSafeMode( + HdfsConstants.SafeModeAction action, boolean isChecked) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + // Set safe mode in all the name spaces + RemoteMethod method = new RemoteMethod("setSafeMode", + new Class[] {HdfsConstants.SafeModeAction.class, boolean.class}, + action, isChecked); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent( + nss, method, true, !isChecked, Boolean.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results + = (Map) o; + // We only report true if all the name space are in safe mode + int numSafemode = 0; + for (boolean safemode : results.values()) { + if (safemode) { + numSafemode++; + } + } + return numSafemode == results.size(); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Boolean.class); + } + + @Override + public boolean saveNamespace(long timeWindow, long txGap) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + RemoteMethod method = new RemoteMethod("saveNamespace", + new Class[] {long.class, long.class}, timeWindow, txGap); + final Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, + false, boolean.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map ret = + (Map) o; + boolean success = true; + for (boolean s : ret.values()) { + if (!s) { + success = false; + break; + } + } + return success; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Boolean.class); + } + + @Override + public long rollEdits() throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + RemoteMethod method = new RemoteMethod("rollEdits", new Class[] {}); + final Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, false, long.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map ret = + (Map) o; + // Return the maximum txid + long txid = 0; + for (long t : ret.values()) { + if (t > txid) { + txid = t; + } + } + return txid; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Long.class); + } + + @Override + public boolean restoreFailedStorage(String arg) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + RemoteMethod method = new RemoteMethod("restoreFailedStorage", + new Class[] {String.class}, arg); + final Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, false, Boolean.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map ret = + (Map) o; + boolean success = true; + for (boolean s : ret.values()) { + if (!s) { + success = false; + break; + } + } + return success; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Boolean.class); + } + + @Override + public RollingUpgradeInfo rollingUpgrade( + HdfsConstants.RollingUpgradeAction action) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("rollingUpgrade", + new Class[] {HdfsConstants.RollingUpgradeAction.class}, action); + final Set nss = namenodeResolver.getNamespaces(); + + rpcClient.invokeConcurrent( + nss, method, true, false, RollingUpgradeInfo.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map ret = + (Map) o; + // Return the first rolling upgrade info + RollingUpgradeInfo info = null; + for (RollingUpgradeInfo infoNs : ret.values()) { + if (info == null && infoNs != null) { + info = infoNs; + } + } + return info; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(RollingUpgradeInfo.class); + } + + @Override + public HdfsFileStatus getFileInfo(String src) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + FileSubclusterResolver subclusterResolver = getSubclusterResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + IOException noLocationException = null; + try { + final List locations = + rpcServer.getLocationsForPath(src, false, false); + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + + // If it's a directory, we check in all locations + if (rpcServer.isPathAll(src)) { + getFileInfoAll(locations, method); + } else { + // Check for file information sequentially + rpcClient.invokeSequential(locations, + method, HdfsFileStatus.class, null); + } + } catch (NoLocationException | RouterResolveException e) { + noLocationException = e; + } + + CompletableFuture completableFuture = getCompletableFuture(); + IOException finalNoLocationException = noLocationException; + completableFuture = completableFuture.thenApply(o -> { + HdfsFileStatus ret = (HdfsFileStatus) o; + // If there is no real path, check mount points + if (ret == null) { + List children = null; + try { + children = subclusterResolver.getMountPoints(src); + } catch (IOException e) { + throw new CompletionException(e); + } + if (children != null && !children.isEmpty()) { + Map dates = getMountPointDates(src); + long date = 0; + if (dates != null && dates.containsKey(src)) { + date = dates.get(src); + } + ret = getMountPointStatus(src, children.size(), date, false); + } else if (children != null) { + // The src is a mount point, but there are no files or directories + ret = getMountPointStatus(src, 0, 0, false); + } + } + + // Can't find mount point for path and the path didn't contain any sub monit points, + // throw the NoLocationException to client. + if (ret == null && finalNoLocationException != null) { + throw new CompletionException(finalNoLocationException); + } + + return ret; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(HdfsFileStatus.class); + } + + private HdfsFileStatus getFileInfoAll( + final List locations, + final RemoteMethod method) throws IOException { + return getFileInfoAll(locations, method, -1); + } + + @Override + public ContentSummary getContentSummary(String path) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + // Get the summaries from regular files + final Collection summaries = new ArrayList<>(); + final List locations = getLocationsForContentSummary(path); + final RemoteMethod method = new RemoteMethod("getContentSummary", + new Class[] {String.class}, new RemoteParam()); + rpcClient.invokeConcurrent(locations, method, + false, -1, ContentSummary.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + final List> results = + (List>) o; + FileNotFoundException notFoundException = null; + for (RemoteResult result : results) { + if (result.hasException()) { + IOException ioe = result.getException(); + if (ioe instanceof FileNotFoundException) { + notFoundException = (FileNotFoundException)ioe; + } else if (!isAllowPartialList()) { + throw new CompletionException(ioe); + } + } else if (result.getResult() != null) { + summaries.add(result.getResult()); + } + } + + // Throw original exception if no original nor mount points + if (summaries.isEmpty() && notFoundException != null) { + throw new CompletionException(notFoundException); + } + + return aggregateContentSummary(summaries); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(ContentSummary.class); + } + + @Override + public void setQuota( + String path, long namespaceQuota, long storagespaceQuota, + StorageType type) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.getQuotaModule() + .setQuota(path, namespaceQuota, storagespaceQuota, type, true); + } + + @Override + public QuotaUsage getQuotaUsage(String path) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + return rpcServer.getQuotaModule().getQuotaUsage(path); + } + + // todo + @Override + public Token getDelegationToken(Text renewer) throws IOException { + return super.getDelegationToken(renewer); + } + + //todo + @Override + public long renewDelegationToken(Token token) throws IOException { + return super.renewDelegationToken(token); + } + + @Override + public void cancelDelegationToken(Token token) throws IOException { + super.cancelDelegationToken(token); + } + + @Override + public String createSnapshot( + String snapshotRoot, String snapshotName) throws IOException { + return asyncSnapshotProto.createSnapshot(snapshotRoot, snapshotName); + } + + @Override + public void deleteSnapshot( + String snapshotRoot, String snapshotName) throws IOException { + asyncSnapshotProto.deleteSnapshot(snapshotRoot, snapshotName); + } + + + @Override + public void allowSnapshot(String snapshotRoot) throws IOException { + asyncSnapshotProto.allowSnapshot(snapshotRoot); + } + + @Override + public void disallowSnapshot(String snapshot) throws IOException { + asyncSnapshotProto.disallowSnapshot(snapshot); + } + + @Override + public void renameSnapshot( + String snapshotRoot, String snapshotOldName, + String snapshotNewName) throws IOException { + asyncSnapshotProto.renameSnapshot( + snapshotRoot, snapshotOldName, snapshotNewName); + } + + @Override + public SnapshottableDirectoryStatus[] getSnapshottableDirListing() + throws IOException { + return asyncSnapshotProto.getSnapshottableDirListing(); + } + + @Override + public SnapshotStatus[] getSnapshotListing(String snapshotRoot) + throws IOException { + return asyncSnapshotProto.getSnapshotListing(snapshotRoot); + } + + @Override + public SnapshotDiffReport getSnapshotDiffReport( + String snapshotRoot, + String earlierSnapshotName, String laterSnapshotName) throws IOException { + return asyncSnapshotProto.getSnapshotDiffReport( + snapshotRoot, earlierSnapshotName, laterSnapshotName); + } + + @Override + public SnapshotDiffReportListing getSnapshotDiffReportListing( + String snapshotRoot, String earlierSnapshotName, String laterSnapshotName, + byte[] startPath, int index) throws IOException { + return asyncSnapshotProto.getSnapshotDiffReportListing( + snapshotRoot, earlierSnapshotName, laterSnapshotName, startPath, index); + } + + @Override + public List getXAttrs(String src, List xAttrs) throws IOException { + RouterRpcClient rpcClient = getRpcClient(); + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + // TODO handle virtual directories + final List locations = + rpcServer.getLocationsForPath(src, false, false); + RemoteMethod method = new RemoteMethod("getXAttrs", + new Class[] {String.class, List.class}, new RemoteParam(), xAttrs); + rpcClient.invokeSequential(locations, method, List.class, null); + return asyncReturn(List.class); + } + + @Override + public List listXAttrs(String src) throws IOException { + RouterRpcClient rpcClient = getRpcClient(); + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + // TODO handle virtual directories + final List locations = + rpcServer.getLocationsForPath(src, false, false); + RemoteMethod method = new RemoteMethod("listXAttrs", + new Class[] {String.class}, new RemoteParam()); + rpcClient.invokeSequential(locations, method, List.class, null); + return asyncReturn(List.class); + } + + @Override + public long getCurrentEditLogTxid() throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod( + "getCurrentEditLogTxid", new Class[] {}); + final Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, false, long.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map ret = + (Map) o; + // Return the maximum txid + long txid = 0; + for (long t : ret.values()) { + if (t > txid) { + txid = t; + } + } + return txid; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Long.class); + } + + @Override + public Path getEnclosingRoot(String src) throws IOException { + Path mountPath = null; + if (isDefaultNameServiceEnabled()) { + mountPath = new Path("/"); + } + + FileSubclusterResolver subclusterResolver = getSubclusterResolver(); + if (subclusterResolver instanceof MountTableResolver) { + MountTableResolver mountTable = (MountTableResolver) subclusterResolver; + if (mountTable.getMountPoint(src) != null) { + mountPath = new Path(mountTable.getMountPoint(src).getSourcePath()); + } + } + + if (mountPath == null) { + throw new IOException(String.format("No mount point for %s", src)); + } + + getEZForPath(src); + CompletableFuture completableFuture = getCompletableFuture(); + Path finalMountPath = mountPath; + completableFuture = completableFuture.thenApply(o -> { + EncryptionZone zone = (EncryptionZone) o; + if (zone == null) { + return finalMountPath; + } else { + Path zonePath = new Path(zone.getPath()); + return zonePath.depth() > finalMountPath.depth() ? zonePath : finalMountPath; + } + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Path.class); + } + + + @Override + public long addCacheDirective(CacheDirectiveInfo path, + EnumSet flags) throws IOException { + return routerAsyncCacheAdmin.addCacheDirective(path, flags); + } + + @Override + public void modifyCacheDirective(CacheDirectiveInfo directive, + EnumSet flags) throws IOException { + routerAsyncCacheAdmin.modifyCacheDirective(directive, flags); + } + + @Override + public void removeCacheDirective(long id) throws IOException { + routerAsyncCacheAdmin.removeCacheDirective(id); + } + + @Override + public BatchedRemoteIterator.BatchedEntries listCacheDirectives( + long prevId, + CacheDirectiveInfo filter) throws IOException { + return routerAsyncCacheAdmin.listCacheDirectives(prevId, filter); + } + + @Override + public void addCachePool(CachePoolInfo info) throws IOException { + routerAsyncCacheAdmin.addCachePool(info); + } + + @Override + public void modifyCachePool(CachePoolInfo info) throws IOException { + routerAsyncCacheAdmin.modifyCachePool(info); + } + + @Override + public void removeCachePool(String cachePoolName) throws IOException { + routerAsyncCacheAdmin.removeCachePool(cachePoolName); + } + + @Override + public BatchedRemoteIterator.BatchedEntries listCachePools(String prevKey) + throws IOException { + return routerAsyncCacheAdmin.listCachePools(prevKey); + } + + @Override + public void setStoragePolicy(String src, String policyName) + throws IOException { + asyncstoragePolicy.setStoragePolicy(src, policyName); + } + + @Override + public BlockStoragePolicy[] getStoragePolicies() throws IOException { + return asyncstoragePolicy.getStoragePolicies(); + } + + + @Override + public void unsetStoragePolicy(String src) throws IOException { + asyncstoragePolicy.unsetStoragePolicy(src); + } + + @Override + public BlockStoragePolicy getStoragePolicy(String path) throws IOException { + return asyncstoragePolicy.getStoragePolicy(path); + } + + @Override + public void satisfyStoragePolicy(String path) throws IOException { + asyncstoragePolicy.satisfyStoragePolicy(path); + } + + @Override + public DatanodeInfo[] getSlowDatanodeReport() throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + return rpcServer.getSlowDatanodeReportAsync(true, 0); + } + + @Override + public boolean setReplication(String src, short replication) + throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + List locations = rpcServer.getLocationsForPath(src, true); + RemoteMethod method = new RemoteMethod("setReplication", + new Class[] {String.class, short.class}, new RemoteParam(), + replication); + if (rpcServer.isInvokeConcurrent(src)) { + rpcClient.invokeConcurrent(locations, method, Boolean.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results = (Map) o; + return !results.containsValue(false); + }); + setCurCompletableFuture(completableFuture); + } else { + rpcClient.invokeSequential(locations, method, Boolean.class, + Boolean.TRUE); + } + return false; + } + + public boolean isMultiDestDirectory(String src) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + try { + if (rpcServer.isPathAll(src)) { + List locations; + locations = rpcServer.getLocationsForPath(src, false, false); + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + rpcClient.invokeSequential(locations, + method, HdfsFileStatus.class, null); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + HdfsFileStatus fileStatus = (HdfsFileStatus) o; + if (fileStatus != null) { + return fileStatus.isDirectory(); + } else { + LOG.debug("The destination {} doesn't exist.", src); + } + return false; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Boolean.class); + } + } catch (UnresolvedPathException e) { + LOG.debug("The destination {} is a symlink.", src); + } + setCurCompletableFuture(CompletableFuture.completedFuture(false)); + return asyncReturn(Boolean.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncNamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncNamenodeProtocol.java new file mode 100644 index 0000000000000..33e06120c9a1a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncNamenodeProtocol.java @@ -0,0 +1,152 @@ +/** + * 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.server.federation.router; + +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; +import org.apache.hadoop.hdfs.server.namenode.NNStorage; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; +import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; +import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.function.Function; + +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.getCompletableFuture; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.setCurCompletableFuture; + +public class RouterAsyncNamenodeProtocol extends RouterNamenodeProtocol{ + /** RPC server to receive client calls. */ + private final RouterRpcServer rpcServer; + /** RPC clients to connect to the Namenodes. */ + private final RouterRpcClient rpcClient; + + public RouterAsyncNamenodeProtocol(RouterRpcServer server) { + super(server); + this.rpcServer = server; + this.rpcClient = this.rpcServer.getRPCClient(); + } + + @Override + public BlocksWithLocations getBlocks( + DatanodeInfo datanode, long size, + long minBlockSize, long hotBlockTimeInterval, StorageType storageType) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + // Get the namespace where the datanode is located + rpcServer.getDatanodeStorageReportMapAsync(HdfsConstants.DatanodeReportType.ALL); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply((Function) o -> { + String nsId = null; + Map map = (Map) o; + for (Map.Entry entry : map.entrySet()) { + DatanodeStorageReport[] dns = entry.getValue(); + for (DatanodeStorageReport dn : dns) { + DatanodeInfo dnInfo = dn.getDatanodeInfo(); + if (dnInfo.getDatanodeUuid().equals(datanode.getDatanodeUuid())) { + nsId = entry.getKey(); + break; + } + } + // Break the loop if already found + if (nsId != null) { + break; + } + } + return nsId; + }).thenCompose(o -> { + // Forward to the proper namenode + if (o != null) { + String nsId = (String) o; + try { + RemoteMethod method = new RemoteMethod( + NamenodeProtocol.class, "getBlocks", new Class[] + {DatanodeInfo.class, long.class, long.class, long.class, StorageType.class}, + datanode, size, minBlockSize, hotBlockTimeInterval, storageType); + rpcClient.invokeSingle(nsId, method, BlocksWithLocations.class); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + } + return CompletableFuture.completedFuture(null); + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(BlocksWithLocations.class); + } + + @Override + public ExportedBlockKeys getBlockKeys() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = + new RemoteMethod(NamenodeProtocol.class, "getBlockKeys"); + rpcServer.invokeAtAvailableNsAsync(method, ExportedBlockKeys.class); + return asyncReturn(ExportedBlockKeys.class); + } + + @Override + public long getTransactionID() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = + new RemoteMethod(NamenodeProtocol.class, "getTransactionID"); + rpcServer.invokeAtAvailableNsAsync(method, long.class); + return asyncReturn(Long.class); + } + + @Override + public long getMostRecentCheckpointTxId() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = + new RemoteMethod(NamenodeProtocol.class, "getMostRecentCheckpointTxId"); + rpcServer.invokeAtAvailableNsAsync(method, long.class); + return asyncReturn(Long.class); + } + + @Override + public long getMostRecentNameNodeFileTxId(NNStorage.NameNodeFile nnf) + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = + new RemoteMethod(NamenodeProtocol.class, "getMostRecentNameNodeFileTxId", + new Class[] {NNStorage.NameNodeFile.class}, nnf); + rpcServer.invokeAtAvailableNsAsync(method, long.class); + return asyncReturn(Long.class); + } + + @Override + public NamespaceInfo versionRequest() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = + new RemoteMethod(NamenodeProtocol.class, "versionRequest"); + rpcServer.invokeAtAvailableNsAsync(method, NamespaceInfo.class); + return asyncReturn(NamespaceInfo.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncRpcClient.java new file mode 100644 index 0000000000000..5ed138c9b4632 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncRpcClient.java @@ -0,0 +1,719 @@ +/** + * 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.server.federation.router; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.NameNodeProxiesClient; +import org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil; +import org.apache.hadoop.hdfs.server.federation.fairness.RouterRpcFairnessPolicyController; +import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.ipc.CallerContext; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.ObserverRetryOnActiveException; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.ipc.RetriableException; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.ipc.StandbyException; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.function.BiFunction; + +import static org.apache.hadoop.hdfs.server.federation.fairness.RouterRpcFairnessConstants.CONCURRENT_NS; +import static org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCPerformanceMonitor.CONCURRENT; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.CUR_COMPLETABLE_FUTURE; + +public class RouterAsyncRpcClient extends RouterRpcClient{ + private static final Logger LOG = + LoggerFactory.getLogger(RouterAsyncRpcClient.class); + + /** + * Create a router RPC client to manage remote procedure calls to NNs. + * + * @param conf Hdfs Configuration. + * @param router A router using this RPC client. + * @param resolver A NN resolver to determine the currently active NN in HA. + * @param monitor Optional performance monitor. + * @param routerStateIdContext the router state context object to hold the state ids for all + * namespaces. + */ + public RouterAsyncRpcClient( + Configuration conf, Router router, ActiveNamenodeResolver resolver, + RouterRpcMonitor monitor, RouterStateIdContext routerStateIdContext) { + super(conf, router, resolver, monitor, routerStateIdContext); + } + + @Override + public boolean invokeAll( + final Collection locations, final RemoteMethod method) + throws IOException { + invokeConcurrent(locations, method, false, false, Boolean.class); + CompletableFuture completableFuture = CUR_COMPLETABLE_FUTURE.get(); + completableFuture = completableFuture.thenApply(o -> { + Map results = (Map) o; + return results.containsValue(true); + }); + CUR_COMPLETABLE_FUTURE.set(completableFuture); +// return (boolean) getResult(); + return false; + } + + @Override + public Object invokeMethod( + UserGroupInformation ugi, + List namenodes, + boolean useObserver, Class protocol, + Method method, Object... params) throws IOException { + LOG.info("{} zj test : {}, {}, {}, {}", Thread.currentThread(), method.getName(), useObserver, namenodes.toString()); + CompletableFuture completableFuture = + invokeMethodAsync(ugi, namenodes, useObserver, protocol, method, params); + CUR_COMPLETABLE_FUTURE.set(completableFuture); + return completableFuture; + } + + private CompletableFuture invokeMethodAsync( + final UserGroupInformation ugi, + final List namenodes, + boolean useObserver, + final Class protocol, final Method method, final Object... params) + throws IOException { + + if (namenodes == null || namenodes.isEmpty()) { + throw new IOException("No namenodes to invoke " + method.getName() + + " with params " + Arrays.deepToString(params) + " from " + + router.getRouterId()); + } + + addClientInfoToCallerContext(ugi); + if (rpcMonitor != null) { + rpcMonitor.incrProcessingOp(); + } + // transfer originCall & callerContext to worker threads of executor. + final Server.Call originCall = Server.getCurCall().get(); + final CallerContext originContext = CallerContext.getCurrent(); + + final long startProxyTime = Time.monotonicNow(); + Map ioes = new LinkedHashMap<>(); + + CompletableFuture completableFuture = + CompletableFuture.completedFuture(new Object[]{useObserver, false, false, null}); + + for (FederationNamenodeContext namenode : namenodes) { + completableFuture = completableFuture.thenCompose(args -> { + LOG.info("{} invokeAsyncTask {} {} {}", + Thread.currentThread(), namenode, method.getName(), params); + Boolean shouldUseObserver = (Boolean) args[0]; + Boolean failover = (Boolean) args[1]; + Boolean complete = (Boolean) args[2]; + if (complete) { + return CompletableFuture.completedFuture( + new Object[]{shouldUseObserver, failover, complete, args[3]}); + } + return invokeAsyncTask(originCall, originContext, startProxyTime, ioes, ugi, + namenode, shouldUseObserver, failover, protocol, method, params); + }); + } + + return completableFuture.thenApply(args -> { + Boolean complete = (Boolean) args[2]; + if (complete) { + return args[3]; + } + // All namenodes were unavailable or in standby + String msg = "No namenode available to invoke " + method.getName() + " " + + Arrays.deepToString(params) + " in " + namenodes + " from " + + router.getRouterId(); + LOG.error(msg); + int exConnect = 0; + for (Map.Entry entry : + ioes.entrySet()) { + FederationNamenodeContext namenode = entry.getKey(); + String nnKey = namenode.getNamenodeKey(); + String addr = namenode.getRpcAddress(); + IOException ioe = entry.getValue(); + if (ioe instanceof StandbyException) { + LOG.error("{} at {} is in Standby: {}", + nnKey, addr, ioe.getMessage()); + } else if (isUnavailableException(ioe)) { + exConnect++; + LOG.error("{} at {} cannot be reached: {}", + nnKey, addr, ioe.getMessage()); + } else { + LOG.error("{} at {} error: \"{}\"", nnKey, addr, ioe.getMessage()); + } + } + if (exConnect == ioes.size()) { + throw new CompletionException(new ConnectException(msg)); + } else { + throw new CompletionException(new StandbyException(msg)); + } + }); + } + + @SuppressWarnings("checkstyle:ParameterNumber") + private CompletableFuture invokeAsyncTask( + final Server.Call originCall, + final CallerContext callerContext, + final long startProxyTime, + final Map ioes, + final UserGroupInformation ugi, + FederationNamenodeContext namenode, + boolean useObserver, + boolean failover, + final Class protocol, final Method method, final Object... params) { + transferThreadLocalContext(originCall, callerContext); + if (!useObserver && (namenode.getState() == FederationNamenodeServiceState.OBSERVER)) { + return CompletableFuture.completedFuture(new Object[]{useObserver, failover, false, null}); + } + String nsId = namenode.getNameserviceId(); + String rpcAddress = namenode.getRpcAddress(); + try { + ConnectionContext connection = getConnection(ugi, nsId, rpcAddress, protocol); + NameNodeProxiesClient.ProxyAndInfo client = connection.getClient(); + return invokeAsync(originCall, callerContext, nsId, namenode, useObserver, + 0, method, client.getProxy(), params) + .handle((result, e) -> { + connection.release(); + boolean complete = false; + if (result != null || e == null) { + complete = true; + if (failover && + FederationNamenodeServiceState.OBSERVER != namenode.getState()) { + // Success on alternate server, update + InetSocketAddress address = client.getAddress(); + try { + namenodeResolver.updateActiveNamenode(nsId, address); + } catch (IOException ex) { + throw new CompletionException(ex); + } + } + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpComplete( + true, nsId, namenode.getState(), Time.monotonicNow() - startProxyTime); + } + if (this.router.getRouterClientMetrics() != null) { + this.router.getRouterClientMetrics().incInvokedMethod(method); + } + return new Object[] {useObserver, failover, complete, result}; + } + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + IOException ioe = (IOException) cause; + ioes.put(namenode, ioe); + if (ioe instanceof ObserverRetryOnActiveException) { + LOG.info("Encountered ObserverRetryOnActiveException from {}." + + " Retry active namenode directly.", namenode); + return new Object[]{false, failover, complete, null}; + } else if (ioe instanceof StandbyException) { + // Fail over indicated by retry policy and/or NN + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpFailureStandby(nsId); + } + return new Object[]{useObserver, true, complete, null}; + } else if (isUnavailableException(ioe)) { + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpFailureCommunicate(nsId); + } + boolean tmpFailover = failover; + if (FederationNamenodeServiceState.OBSERVER == namenode.getState()) { + try { + namenodeResolver.updateUnavailableNamenode(nsId, + NetUtils.createSocketAddr(namenode.getRpcAddress())); + } catch (IOException ex) { + throw new CompletionException(ex); + } + } else { + tmpFailover = true; + } + return new Object[]{useObserver, tmpFailover, complete, null}; + } else if (ioe instanceof RemoteException) { + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpComplete( + true, nsId, namenode.getState(), Time.monotonicNow() - startProxyTime); + } + RemoteException re = (RemoteException) ioe; + ioe = re.unwrapRemoteException(); + ioe = getCleanException(ioe); + // RemoteException returned by NN + throw new CompletionException(ioe); + } else if (ioe instanceof NoNamenodesAvailableException) { + IOException cau = (IOException) ioe.getCause(); + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpNoNamenodes(nsId); + } + LOG.error("Cannot get available namenode for {} {} error: {}", + nsId, rpcAddress, ioe.getMessage()); + // Rotate cache so that client can retry the next namenode in the cache + if (shouldRotateCache(cau)) { + this.namenodeResolver.rotateCache(nsId, namenode, useObserver); + } + // Throw RetriableException so that client can retry + throw new CompletionException(new RetriableException(ioe)); + } else { + // Other communication error, this is a failure + // Communication retries are handled by the retry policy + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpFailureCommunicate(nsId); + this.rpcMonitor.proxyOpComplete( + false, nsId, namenode.getState(), Time.monotonicNow() - startProxyTime); + } + throw new CompletionException(ioe); + } + } + throw new CompletionException(cause); + }); + }catch (IOException ioe) { + assert ioe instanceof ConnectionNullException; + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpFailureCommunicate(nsId); + } + LOG.error("Get connection for {} {} error: {}", nsId, rpcAddress, + ioe.getMessage()); + // Throw StandbyException so that client can retry + StandbyException se = new StandbyException(ioe.getMessage()); + se.initCause(ioe); + throw new CompletionException(se); + } + } + + + @SuppressWarnings("checkstyle:ParameterNumber") + private CompletableFuture invokeAsync( + final Server.Call originCall, + final CallerContext callerContext, + String nsId, FederationNamenodeContext namenode, + Boolean listObserverFirst, + int retryCount, final Method method, + final Object obj, final Object... params) { + try { + transferThreadLocalContext(originCall, callerContext); + Client.setAsynchronousMode(true); + method.invoke(obj, params); + // TODO: for test. because dfsclient and router may use main thread. + // so unset the value , ensure dfsclient rpc is sync. + Client.setAsynchronousMode(false); + CompletableFuture completableFuture = + AsyncRpcProtocolPBUtil.getCompletableFuture(); + + return completableFuture.handle((BiFunction) (result, e) -> { + if (e == null) { + return new Object[]{result, true}; + } + + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + IOException ioe = (IOException) cause; + + // Check if we should retry. + RetryPolicy.RetryAction.RetryDecision decision = null; + try { + decision = shouldRetry(ioe, retryCount, nsId, namenode, listObserverFirst); + } catch (IOException ex) { + throw new CompletionException(ex); + } + if (decision == RetryPolicy.RetryAction.RetryDecision.RETRY) { + if (RouterAsyncRpcClient.this.rpcMonitor != null) { + RouterAsyncRpcClient.this.rpcMonitor.proxyOpRetries(); + } + // retry + return new Object[]{result, false}; + } else if (decision == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY) { + // failover, invoker looks for standby exceptions for failover. + if (ioe instanceof StandbyException) { + throw new CompletionException(ioe); + } else if (isUnavailableException(ioe)) { + throw new CompletionException(ioe); + } else { + throw new CompletionException(new StandbyException(ioe.getMessage())); + } + } else { + throw new CompletionException(ioe); + } + } else { + throw new CompletionException(new IOException(e)); + } + }).thenCompose(o -> { + Object[] args = (Object[]) o; + boolean complete = (boolean) args[1]; + if (complete) { + return CompletableFuture.completedFuture(args[0]); + } + return invokeAsync(originCall, callerContext, nsId, namenode, + listObserverFirst, retryCount + 1, method, obj, params); + }); + } catch (InvocationTargetException e) { + throw new CompletionException(e.getCause()); + } catch (Exception e) { + throw new CompletionException(e); + } + } + + @Override + public T invokeSequential( + final List locations, + final RemoteMethod remoteMethod, Class expectedResultClass, + Object expectedResultValue) throws IOException { + invokeSequential(remoteMethod, locations, expectedResultClass, expectedResultValue); + CompletableFuture completableFuture = CUR_COMPLETABLE_FUTURE.get(); + completableFuture = completableFuture.thenApply(o -> { + RemoteResult result = (RemoteResult) o; + return result.getResult(); + }); + CUR_COMPLETABLE_FUTURE.set(completableFuture); +// return (T) getResult(); + return RouterAsyncRpcUtil.asyncReturn(expectedResultClass); + } + + @Override + public RemoteResult invokeSequential( + final RemoteMethod remoteMethod, final List locations, + Class expectedResultClass, Object expectedResultValue) + throws IOException { + RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController(); + final UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + final Method m = remoteMethod.getMethod(); + List thrownExceptions = new ArrayList<>(); + List results = new ArrayList<>(); + CompletableFuture completableFuture = + CompletableFuture.completedFuture(new Object[] {null, false}); + // Invoke in priority order + for (final RemoteLocationContext loc : locations) { + String ns = loc.getNameserviceId(); + acquirePermit(ns, ugi, remoteMethod, controller); + completableFuture = completableFuture.thenCompose(args -> { + boolean complete = (boolean) args[1]; + if (complete) { + return CompletableFuture.completedFuture(new Object[]{args[0], true}); + } + return invokeSequentialToOneNs(ugi, m, + thrownExceptions, remoteMethod, loc, expectedResultClass, + expectedResultValue, results); + }); + + releasePermit(ns, ugi, remoteMethod, controller); + } + + CompletableFuture resultFuture = completableFuture.thenApply(args -> { + boolean complete = (boolean) args[1]; + if (complete) { + return args[0]; + } + if (!thrownExceptions.isEmpty()) { + // An unavailable subcluster may be the actual cause + // We cannot surface other exceptions (e.g., FileNotFoundException) + for (int i = 0; i < thrownExceptions.size(); i++) { + IOException ioe = thrownExceptions.get(i); + if (isUnavailableException(ioe)) { + throw new CompletionException(ioe); + } + } + + // re-throw the first exception thrown for compatibility + throw new CompletionException(thrownExceptions.get(0)); + } + // Return the first result, whether it is the value or not + return new RemoteResult<>(locations.get(0), results.get(0)); + }); + System.out.println("zjcom2: " + resultFuture); + CUR_COMPLETABLE_FUTURE.set(resultFuture); +// return (RemoteResult) getResult(); + return null; + } + + @SuppressWarnings("checkstyle:ParameterNumber") + private CompletableFuture invokeSequentialToOneNs( + final UserGroupInformation ugi, final Method m, + final List thrownExceptions, + final RemoteMethod remoteMethod, final RemoteLocationContext loc, + final Class expectedResultClass, final Object expectedResultValue, + final List results) { + String ns = loc.getNameserviceId(); + boolean isObserverRead = isObserverReadEligible(ns, m); + try { + List namenodes = + getOrderedNamenodes(ns, isObserverRead); + Class proto = remoteMethod.getProtocol(); + Object[] params = remoteMethod.getParams(loc); + + CompletableFuture completableFuture = + (CompletableFuture) invokeMethod(ugi, namenodes, + isObserverRead, proto, m, params); + return completableFuture.handle((result, e) -> { + if (e == null) { + // Check if the result is what we expected + if (isExpectedClass(expectedResultClass, result) && + isExpectedValue(expectedResultValue, result)) { + // Valid result, stop here + return new Object[] {new RemoteResult<>(loc, result), true}; + } else { + results.add(result); + return new Object[] {null, false}; + } + } + + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + IOException ioe = (IOException) cause; + // Localize the exception + + ioe = processException(ioe, loc); + + // Record it and move on + thrownExceptions.add(ioe); + } else { + // Unusual error, ClientProtocol calls always use IOException (or + // RemoteException). Re-wrap in IOException for compatibility with + // ClientProtocol. + LOG.error("Unexpected exception {} proxying {} to {}", + e.getClass(), m.getName(), ns, e); + IOException ioe = new IOException( + "Unexpected exception proxying API " + e.getMessage(), e); + thrownExceptions.add(ioe); + } + return new Object[] {null, false}; + }); + }catch (IOException ioe) { + throw new CompletionException(ioe); + } + } + + @Override + public Map invokeConcurrent( + final Collection locations, final RemoteMethod method, + boolean requireResponse, boolean standby, long timeOutMs, Class clazz) + throws IOException { + invokeConcurrentAsync(locations, method, standby, timeOutMs, clazz); + CompletableFuture completableFuture = CUR_COMPLETABLE_FUTURE.get(); + completableFuture = completableFuture.thenApply(o -> { + final List> results = (List>) o; + // Go over the results and exceptions + final Map ret = new TreeMap<>(); + final List thrownExceptions = new ArrayList<>(); + IOException firstUnavailableException = null; + for (final RemoteResult result : results) { + if (result.hasException()) { + IOException ioe = result.getException(); + thrownExceptions.add(ioe); + // Track unavailable exceptions to throw them first + if (isUnavailableException(ioe)) { + firstUnavailableException = ioe; + } + } + if (result.hasResult()) { + ret.put(result.getLocation(), result.getResult()); + } + } + // Throw exceptions if needed + if (!thrownExceptions.isEmpty()) { + // Throw if response from all servers required or no results + if (requireResponse || ret.isEmpty()) { + // Throw unavailable exceptions first + if (firstUnavailableException != null) { + throw new CompletionException(firstUnavailableException); + } else { + throw new CompletionException(thrownExceptions.get(0)); + } + } + } + return ret; + }); + CUR_COMPLETABLE_FUTURE.set(completableFuture); +// return (Map) getResult(); + return null; + } + + @Override + @SuppressWarnings("checkstyle:MethodLength") + public List> + invokeConcurrent(final Collection locations, + final RemoteMethod method, boolean standby, long timeOutMs, + Class clazz) throws IOException { + invokeConcurrentAsync(locations, method, standby, timeOutMs, clazz); +// return (List>) getResult(); + return null; + } + + private List> + invokeConcurrentAsync(final Collection locations, + final RemoteMethod method, boolean standby, long timeOutMs, + Class clazz) throws IOException { + final UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + final Method m = method.getMethod(); + + if (locations.isEmpty()) { + throw new IOException("No remote locations available"); + } else if (locations.size() == 1 && timeOutMs <= 0) { + // Shortcut, just one call + T location = locations.iterator().next(); + String ns = location.getNameserviceId(); + boolean isObserverRead = isObserverReadEligible(ns, m); + final List namenodes = + getOrderedNamenodes(ns, isObserverRead); + RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController(); + acquirePermit(ns, ugi, method, controller); + try { + Class proto = method.getProtocol(); + Object[] paramList = method.getParams(location); + invokeMethod(ugi, namenodes, isObserverRead, proto, m, paramList); + CompletableFuture completableFuture = CUR_COMPLETABLE_FUTURE.get(); + completableFuture = completableFuture.exceptionally(e -> { + IOException ioe = (IOException) e.getCause(); + throw new CompletionException(processException(ioe, location)); + }).thenApply(result -> { + RemoteResult remoteResult = + (RemoteResult) new RemoteResult<>(location, result); + return Collections.singletonList(remoteResult); + }); + CUR_COMPLETABLE_FUTURE.set(completableFuture); + return null; + } catch (IOException ioe) { + // Localize the exception + throw processException(ioe, location); + } finally { + releasePermit(ns, ugi, method, controller); + } + } + + if (rpcMonitor != null) { + rpcMonitor.incrProcessingOp(); + } + if (this.router.getRouterClientMetrics() != null) { + this.router.getRouterClientMetrics().incInvokedConcurrent(m); + } + + RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController(); + acquirePermit(CONCURRENT_NS, ugi, method, controller); + + List orderedLocations = new ArrayList<>(); + List> completableFutures = new ArrayList<>(); + for (final T location : locations) { + String nsId = location.getNameserviceId(); + boolean isObserverRead = isObserverReadEligible(nsId, m); + final List namenodes = + getOrderedNamenodes(nsId, isObserverRead); + final Class proto = method.getProtocol(); + final Object[] paramList = method.getParams(location); + if (standby) { + // Call the objectGetter to all NNs (including standby) + for (final FederationNamenodeContext nn : namenodes) { + final List nnList = + Collections.singletonList(nn); + String nnId = nn.getNamenodeId(); + T nnLocation = location; + if (location instanceof RemoteLocation) { + nnLocation = (T)new RemoteLocation(nsId, nnId, location.getDest()); + } + orderedLocations.add(nnLocation); + invokeMethod(ugi, nnList, isObserverRead, proto, m, paramList); + completableFutures.add(CUR_COMPLETABLE_FUTURE.get()); + } + } else { + // Call the objectGetter in order of nameservices in the NS list + orderedLocations.add(location); + invokeMethod(ugi, namenodes, isObserverRead, proto, m, paramList); + completableFutures.add(CUR_COMPLETABLE_FUTURE.get()); + } + } + + CompletableFuture[] completableFuturesArray = + new CompletableFuture[completableFutures.size()]; + completableFuturesArray = completableFutures.toArray(completableFuturesArray); + CompletableFuture allFuture = CompletableFuture.allOf(completableFuturesArray); + CompletableFuture resultCompletable = allFuture.handle((unused, throwable) -> { + List> results = new ArrayList<>(); + for (int i=0; i resultFuture = completableFutures.get(i); + Object result = null; + try { + result = resultFuture.get(); + results.add((RemoteResult) new RemoteResult<>(location, result)); + } catch (InterruptedException ignored) { + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + IOException ioe = null; + if (cause instanceof CancellationException) { + T loc = orderedLocations.get(i); + String msg = "Invocation to \"" + loc + "\" for \"" + + method.getMethodName() + "\" timed out"; + LOG.error(msg); + ioe = new SubClusterTimeoutException(msg); + } else if (cause instanceof IOException) { + LOG.debug("Cannot execute {} in {}: {}", + m.getName(), location, cause.getMessage()); + ioe = (IOException) cause; + } else { + ioe = new IOException("Unhandled exception while proxying API " + + m.getName() + ": " + cause.getMessage(), cause); + } + // Store the exceptions + results.add(new RemoteResult<>(location, ioe)); + } + } + if (rpcMonitor != null) { + rpcMonitor.proxyOpComplete(true, CONCURRENT, null); + } + return results; + }); + + CUR_COMPLETABLE_FUTURE.set(resultCompletable); + releasePermit(CONCURRENT_NS, ugi, method, controller); + return null; + } + + @Override + public T invokeSingle(final RemoteLocationContext location, + RemoteMethod remoteMethod, Class clazz) throws IOException { + List locations = Collections.singletonList(location); + invokeSequential(locations, remoteMethod); +// return (T) getResult(); + return RouterAsyncRpcUtil.asyncReturn(clazz); + } + + @Override + public Object invokeSingle(final String nsId, RemoteMethod method) + throws IOException { + super.invokeSingle(nsId, method); +// return getResult(); + return null; + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncRpcUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncRpcUtil.java new file mode 100644 index 0000000000000..c0b211febb375 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncRpcUtil.java @@ -0,0 +1,83 @@ +/** + * 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.server.federation.router; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; + +public final class RouterAsyncRpcUtil { + public static final ThreadLocal> CUR_COMPLETABLE_FUTURE + = new ThreadLocal<>(); + private static final Boolean BOOLEAN_RESULT = false; + private static final Long LONG_RESULT = -1L; + private static final Object NULL_RESULT = null; + + private RouterAsyncRpcUtil(){} + + public static CompletableFuture getCompletableFuture() { + return CUR_COMPLETABLE_FUTURE.get(); + } + + public static void setCurCompletableFuture( + CompletableFuture completableFuture) { + CUR_COMPLETABLE_FUTURE.set(completableFuture); + } + + public static Object getResult() throws IOException { + try { + CompletableFuture completableFuture = CUR_COMPLETABLE_FUTURE.get(); + Object o = completableFuture.get(); + return o; + } catch (InterruptedException e) { + } catch (ExecutionException e) { + IOException ioe = (IOException) e.getCause(); + throw ioe; + } + return null; + } + + public static T asyncReturn(Class clazz) { + if (clazz == null) { + return null; + } + if (clazz.equals(Boolean.class)) { + return (T) BOOLEAN_RESULT; + } else if (clazz.equals(Long.class)) { + return (T) LONG_RESULT; + } + return (T) NULL_RESULT; + } + + public static T asyncRequestThenApply(AsyncRequest asyncRequest, + Function thenDo, Class clazz) throws IOException { + asyncRequest.res(); + CompletableFuture completableFuture = + (CompletableFuture) CUR_COMPLETABLE_FUTURE.get(); + CompletableFuture resCompletableFuture = completableFuture.thenApply(thenDo); + setCurCompletableFuture((CompletableFuture) resCompletableFuture); + return asyncReturn(clazz); + } + + @FunctionalInterface + interface AsyncRequest { + R res() throws IOException; + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncSnapshot.java new file mode 100644 index 0000000000000..11a0a3de7a5d6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncSnapshot.java @@ -0,0 +1,193 @@ +/** + * 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.server.federation.router; + +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing; +import org.apache.hadoop.hdfs.protocol.SnapshotStatus; +import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; +import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.asyncRequestThenApply; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.asyncReturn; + +public class RouterAsyncSnapshot extends RouterSnapshot{ + public RouterAsyncSnapshot(RouterRpcServer server) { + super(server); + } + + public String createSnapshot(String snapshotRoot, String snapshotName) + throws IOException { + RouterRpcClient rpcClient = getRpcClient(); + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + final List locations = + rpcServer.getLocationsForPath(snapshotRoot, true, false); + RemoteMethod method = new RemoteMethod("createSnapshot", + new Class[] {String.class, String.class}, new RemoteParam(), + snapshotName); + + if (rpcServer.isInvokeConcurrent(snapshotRoot)) { + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent(locations, method, String.class), + results -> { + Map.Entry firstelement = + results.entrySet().iterator().next(); + RemoteLocation loc = firstelement.getKey(); + String result = firstelement.getValue(); + result = result.replaceFirst(loc.getDest(), loc.getSrc()); + return result; + }, String.class); + } else { + return asyncRequestThenApply( + () -> rpcClient.invokeSequential(method, locations, + String.class, null), + response -> { + RemoteLocation loc = (RemoteLocation) response.getLocation(); + String invokedResult = (String) response.getResult(); + return invokedResult.replaceFirst(loc.getDest(), loc.getSrc()); + }, String.class); + } + } + + public SnapshottableDirectoryStatus[] getSnapshottableDirListing() + throws IOException { + RouterRpcClient rpcClient = getRpcClient(); + RouterRpcServer rpcServer = getRpcServer(); + ActiveNamenodeResolver namenodeResolver = getNamenodeResolver(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getSnapshottableDirListing"); + Set nss = namenodeResolver.getNamespaces(); + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent(nss, method, true, + false, SnapshottableDirectoryStatus[].class), + ret -> RouterRpcServer.merge(ret, SnapshottableDirectoryStatus.class), + SnapshottableDirectoryStatus[].class); + } + + public SnapshotStatus[] getSnapshotListing(String snapshotRoot) + throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + final List locations = + rpcServer.getLocationsForPath(snapshotRoot, true, false); + RemoteMethod remoteMethod = new RemoteMethod("getSnapshotListing", + new Class[]{String.class}, + new RemoteParam()); + + if (rpcServer.isInvokeConcurrent(snapshotRoot)) { + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent(locations, remoteMethod, true, + false, SnapshotStatus[].class), + ret -> { + SnapshotStatus[] response = ret.values().iterator().next(); + String src = ret.keySet().iterator().next().getSrc(); + String dst = ret.keySet().iterator().next().getDest(); + for (SnapshotStatus s : response) { + String mountPath = DFSUtil.bytes2String(s.getParentFullPath()). + replaceFirst(src, dst); + s.setParentFullPath(DFSUtil.string2Bytes(mountPath)); + } + return response; + }, SnapshotStatus[].class); + } else { + return asyncRequestThenApply( + () -> rpcClient.invokeSequential(remoteMethod, locations, + SnapshotStatus[].class, null), + invokedResponse -> { + RemoteLocation loc = (RemoteLocation) invokedResponse.getLocation(); + SnapshotStatus[] response = (SnapshotStatus[]) invokedResponse.getResult(); + for (SnapshotStatus s : response) { + String mountPath = DFSUtil.bytes2String(s.getParentFullPath()). + replaceFirst(loc.getDest(), loc.getSrc()); + s.setParentFullPath(DFSUtil.string2Bytes(mountPath)); + } + return response; + }, SnapshotStatus[].class); + } + } + + public SnapshotDiffReport getSnapshotDiffReport( + String snapshotRoot, + String earlierSnapshotName, String laterSnapshotName) throws IOException { + RouterRpcClient rpcClient = getRpcClient(); + RouterRpcServer rpcServer = getRpcServer(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + final List locations = + rpcServer.getLocationsForPath(snapshotRoot, true, false); + RemoteMethod remoteMethod = new RemoteMethod("getSnapshotDiffReport", + new Class[] {String.class, String.class, String.class}, + new RemoteParam(), earlierSnapshotName, laterSnapshotName); + + if (rpcServer.isInvokeConcurrent(snapshotRoot)) { + return asyncRequestThenApply( + () -> rpcClient.invokeConcurrent(locations, remoteMethod, + true, false, SnapshotDiffReport.class), + ret -> ret.values().iterator().next(), SnapshotDiffReport.class); + } else { + rpcClient.invokeSequential( + locations, remoteMethod, SnapshotDiffReport.class, null); + return asyncReturn(SnapshotDiffReport.class); + } + } + + public SnapshotDiffReportListing getSnapshotDiffReportListing( + String snapshotRoot, String earlierSnapshotName, String laterSnapshotName, + byte[] startPath, int index) throws IOException { + RouterRpcServer rpcServer = getRpcServer(); + RouterRpcClient rpcClient = getRpcClient(); + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + final List locations = + rpcServer.getLocationsForPath(snapshotRoot, true, false); + Class[] params = new Class[] { + String.class, String.class, String.class, + byte[].class, int.class}; + RemoteMethod remoteMethod = new RemoteMethod( + "getSnapshotDiffReportListing", params, + new RemoteParam(), earlierSnapshotName, laterSnapshotName, + startPath, index); + + if (rpcServer.isInvokeConcurrent(snapshotRoot)) { + return asyncRequestThenApply(() -> rpcClient.invokeConcurrent(locations, remoteMethod, + false, false, SnapshotDiffReportListing.class), + ret -> { + Collection listings = ret.values(); + return listings.iterator().next(); + }, SnapshotDiffReportListing.class); + } else { + rpcClient.invokeSequential( + locations, remoteMethod, SnapshotDiffReportListing.class, null); + return asyncReturn(SnapshotDiffReportListing.class); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncStoragePolicy.java new file mode 100644 index 0000000000000..af3a68cc578f8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncStoragePolicy.java @@ -0,0 +1,26 @@ +package org.apache.hadoop.hdfs.server.federation.router; + +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; + +public class RouterAsyncStoragePolicy extends RouterStoragePolicy{ + /** RPC server to receive client calls. */ + private final RouterRpcServer rpcServer; + /** RPC clients to connect to the Namenodes. */ + private final RouterRpcClient rpcClient; + + public RouterAsyncStoragePolicy(RouterRpcServer server) { + super(server); + this.rpcServer = server; + this.rpcClient = this.rpcServer.getRPCClient(); + } + + public BlockStoragePolicy[] getStoragePolicies() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getStoragePolicies"); + return rpcServer.invokeAtAvailableNsAsync(method, BlockStoragePolicy[].class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncUserProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncUserProtocol.java new file mode 100644 index 0000000000000..991ee1447a507 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAsyncUserProtocol.java @@ -0,0 +1,95 @@ +package org.apache.hadoop.hdfs.server.federation.router; + +import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.security.Groups; +import org.apache.hadoop.security.RefreshUserMappingsProtocol; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.ProxyUsers; +import org.apache.hadoop.tools.GetUserMappingsProtocol; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +import static org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.merge; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.getCompletableFuture; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.setCurCompletableFuture; + +public class RouterAsyncUserProtocol extends RouterUserProtocol{ + private static final Logger LOG = + LoggerFactory.getLogger(RouterAsyncUserProtocol.class); + + /** RPC server to receive client calls. */ + private final RouterRpcServer rpcServer; + /** RPC clients to connect to the Namenodes. */ + private final RouterRpcClient rpcClient; + + private final ActiveNamenodeResolver namenodeResolver; + + public RouterAsyncUserProtocol(RouterRpcServer server) { + super(server); + this.rpcServer = server; + this.rpcClient = this.rpcServer.getRPCClient(); + this.namenodeResolver = this.rpcServer.getNamenodeResolver(); + } + + @Override + public void refreshUserToGroupsMappings() throws IOException { + LOG.debug("Refresh user groups mapping in Router."); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + Set nss = namenodeResolver.getNamespaces(); + if (nss.isEmpty()) { + Groups.getUserToGroupsMappingService().refresh(); + setCurCompletableFuture(CompletableFuture.completedFuture(null)); + } else { + RemoteMethod method = new RemoteMethod(RefreshUserMappingsProtocol.class, + "refreshUserToGroupsMappings"); + rpcClient.invokeConcurrent(nss, method); + } + } + + @Override + public void refreshSuperUserGroupsConfiguration() throws IOException { + LOG.debug("Refresh superuser groups configuration in Router."); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + Set nss = namenodeResolver.getNamespaces(); + if (nss.isEmpty()) { + ProxyUsers.refreshSuperUserGroupsConfiguration(); + setCurCompletableFuture(CompletableFuture.completedFuture(null)); + } else { + RemoteMethod method = new RemoteMethod(RefreshUserMappingsProtocol.class, + "refreshSuperUserGroupsConfiguration"); + rpcClient.invokeConcurrent(nss, method); + } + } + + @Override + public String[] getGroupsForUser(String user) throws IOException { + LOG.debug("Getting groups for user {}", user); + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + Set nss = namenodeResolver.getNamespaces(); + if (nss.isEmpty()) { + setCurCompletableFuture( + CompletableFuture + .completedFuture(UserGroupInformation.createRemoteUser(user) + .getGroupNames())); + } else { + RemoteMethod method = new RemoteMethod(GetUserMappingsProtocol.class, + "getGroupsForUser", new Class[] {String.class}, user); + rpcClient.invokeConcurrent(nss, method, String[].class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results = + (Map) o; + return merge(results, String.class); + }); + setCurCompletableFuture(completableFuture); + } + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterCacheAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterCacheAdmin.java index e25d8b269dfb9..11172e9934bb9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterCacheAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterCacheAdmin.java @@ -161,7 +161,7 @@ public BatchedEntries listCachePools(String prevKey) * @param locations the locations to map. * @return map with CacheDirectiveInfo mapped to the locations. */ - private Map getRemoteMap( + Map getRemoteMap( CacheDirectiveInfo path, final List locations) { final Map dstMap = new HashMap<>(); Iterator iterator = locations.iterator(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java index d50648219050f..bfedf2ceb1dbc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java @@ -117,7 +117,11 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CompletionStage; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -347,7 +351,7 @@ protected static boolean isUnavailableSubclusterException( * @throws IOException If this path is not fault tolerant or the exception * should not be retried (e.g., NSQuotaExceededException). */ - private List checkFaultTolerantRetry( + List checkFaultTolerantRetry( final RemoteMethod method, final String src, final IOException ioe, final RemoteLocation excludeLoc, final List locations) throws IOException { @@ -820,7 +824,7 @@ public void renewLease(String clientName, List namespaces) /** * For {@link #getListing(String,byte[],boolean) GetLisiting} to sort results. */ - private static class GetListingComparator + static class GetListingComparator implements Comparator, Serializable { @Override public int compare(byte[] o1, byte[] o2) { @@ -1104,7 +1108,7 @@ public DatanodeStorageReport[] getDatanodeStorageReport( return mergeDtanodeStorageReport(dnSubcluster); } - private DatanodeStorageReport[] mergeDtanodeStorageReport( + DatanodeStorageReport[] mergeDtanodeStorageReport( Map dnSubcluster) { // Avoid repeating machines in multiple subclusters Map datanodesMap = new LinkedHashMap<>(); @@ -1539,11 +1543,13 @@ public void addCachePool(CachePoolInfo info) throws IOException { routerCacheAdmin.addCachePool(info); } + //todo @Override public void modifyCachePool(CachePoolInfo info) throws IOException { routerCacheAdmin.modifyCachePool(info); } + //todo @Override public void removeCachePool(String cachePoolName) throws IOException { routerCacheAdmin.removeCachePool(cachePoolName); @@ -1555,6 +1561,7 @@ public BatchedEntries listCachePools(String prevKey) return routerCacheAdmin.listCachePools(prevKey); } + @Override public void modifyAclEntries(String src, List aclSpec) throws IOException { @@ -1912,17 +1919,20 @@ public ErasureCodingPolicy getErasureCodingPolicy(String src) return erasureCoding.getErasureCodingPolicy(src); } + //todo @Override public void setErasureCodingPolicy(String src, String ecPolicyName) throws IOException { erasureCoding.setErasureCodingPolicy(src, ecPolicyName); } + //todo @Override public void unsetErasureCodingPolicy(String src) throws IOException { erasureCoding.unsetErasureCodingPolicy(src); } + //todo @Override public ECTopologyVerifierResult getECTopologyResultForPolicies( String... policyNames) throws IOException { @@ -1930,6 +1940,7 @@ public ECTopologyVerifierResult getECTopologyResultForPolicies( return erasureCoding.getECTopologyResultForPolicies(policyNames); } + //todo @Override public ECBlockGroupStats getECBlockGroupStats() throws IOException { return erasureCoding.getECBlockGroupStats(); @@ -2039,7 +2050,7 @@ public HAServiceProtocol.HAServiceState getHAServiceState() { * replacement value. * @throws IOException If the dst paths could not be determined. */ - private RemoteParam getRenameDestinations( + RemoteParam getRenameDestinations( final List srcLocations, final List dstLocations) throws IOException { @@ -2087,7 +2098,7 @@ private RemoteLocation getFirstMatchingLocation(RemoteLocation location, * @param summaries Collection of individual summaries. * @return Aggregated content summary. */ - private ContentSummary aggregateContentSummary( + ContentSummary aggregateContentSummary( Collection summaries) { if (summaries.size() == 1) { return summaries.iterator().next(); @@ -2191,7 +2202,7 @@ private HdfsFileStatus getFileInfoAll(final List locations, * @param mask The permission mask of the child. * @return The permission mask of the parent. */ - private static FsPermission getParentPermission(final FsPermission mask) { + static FsPermission getParentPermission(final FsPermission mask) { FsPermission ret = new FsPermission( mask.getUserAction().or(FsAction.WRITE_EXECUTE), mask.getGroupAction(), @@ -2300,7 +2311,7 @@ HdfsFileStatus getMountPointStatus( * @param path Name of the path to start checking dates from. * @return Map with the modification dates for all sub-entries. */ - private Map getMountPointDates(String path) { + Map getMountPointDates(String path) { Map ret = new TreeMap<>(); if (subclusterResolver instanceof MountTableResolver) { try { @@ -2363,7 +2374,7 @@ private long getModifiedTime(Map ret, String path, /** * Get listing on remote locations. */ - private List> getListingInt( + List> getListingInt( String src, byte[] startAfter, boolean needLocation) throws IOException { try { List locations = @@ -2402,7 +2413,7 @@ private List> getListingInt( * @param remainingEntries how many entries left from subcluster * @return */ - private static boolean shouldAddMountPoint( + static boolean shouldAddMountPoint( byte[] mountPoint, byte[] lastEntry, byte[] startAfter, int remainingEntries) { if (comparator.compare(mountPoint, startAfter) > 0 && @@ -2424,8 +2435,7 @@ private static boolean shouldAddMountPoint( * subclusters else false in all other scenarios. * @throws IOException if unable to get the file status. */ - @VisibleForTesting - boolean isMultiDestDirectory(String src) throws IOException { + public boolean isMultiDestDirectory(String src) throws IOException { try { if (rpcServer.isPathAll(src)) { List locations; @@ -2449,4 +2459,60 @@ boolean isMultiDestDirectory(String src) throws IOException { public int getRouterFederationRenameCount() { return rbfRename.getRouterFederationRenameCount(); } + + public RouterRpcServer getRpcServer() { + return rpcServer; + } + + public RouterRpcClient getRpcClient() { + return rpcClient; + } + + public RouterFederationRename getRbfRename() { + return rbfRename; + } + + public ActiveNamenodeResolver getNamenodeResolver() { + return namenodeResolver; + } + + public boolean isAllowPartialList() { + return allowPartialList; + } + + public boolean isDefaultNameServiceEnabled() { + return defaultNameServiceEnabled; + } + + public FileSubclusterResolver getSubclusterResolver() { + return subclusterResolver; + } + + public static GetListingComparator getComparator() { + return comparator; + } + + public String getSuperUser() { + return superUser; + } + + public String getSuperGroup() { + return superGroup; + } + + public long getMountStatusTimeOut() { + return mountStatusTimeOut; + } + + public long getServerDefaultsLastUpdate() { + return serverDefaultsLastUpdate; + } + + public long getServerDefaultsValidityPeriod() { + return serverDefaultsValidityPeriod; + } + + public void setServerDefaultsLastUpdate(long serverDefaultsLastUpdate) { + this.serverDefaultsLastUpdate = serverDefaultsLastUpdate; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java index e9b780d5bca29..3773d27e89168 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java @@ -99,6 +99,9 @@ protected void periodicInvoke() { // This is because mount table does not have mtime. // For other mount entry get current quota usage HdfsFileStatus ret = this.rpcServer.getFileInfo(src); + if (rpcServer.isAsync()) { + ret = (HdfsFileStatus) RouterAsyncRpcUtil.getResult(); + } if (ret == null || ret.getModificationTime() == 0) { long[] zeroConsume = new long[StorageType.values().length]; currentQuotaUsage = @@ -113,6 +116,9 @@ protected void periodicInvoke() { Quota quotaModule = this.rpcServer.getQuotaModule(); Map usageMap = quotaModule.getEachQuotaUsage(src); + if (router.isEnableAsync()) { + usageMap = (Map) RouterAsyncRpcUtil.getResult(); + } currentQuotaUsage = quotaModule.aggregateQuota(src, usageMap); remoteQuotaUsage.putAll(usageMap); } catch (IOException ioe) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index d25e5ae4d3012..52cded2d14e3e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java @@ -50,6 +50,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -66,6 +67,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.SnapshotException; import org.apache.hadoop.hdfs.server.federation.fairness.RouterRpcFairnessPolicyController; @@ -120,10 +122,10 @@ public class RouterRpcClient { /** Router using this RPC client. */ - private final Router router; + protected final Router router; /** Interface to identify the active NN for a nameservice or blockpool ID. */ - private final ActiveNamenodeResolver namenodeResolver; + protected final ActiveNamenodeResolver namenodeResolver; /** Connection pool to the Namenodes per user for performance. */ private final ConnectionManager connectionManager; @@ -132,7 +134,7 @@ public class RouterRpcClient { /** Retry policy for router -> NN communication. */ private final RetryPolicy retryPolicy; /** Optional perf monitor. */ - private final RouterRpcMonitor rpcMonitor; + protected final RouterRpcMonitor rpcMonitor; /** Field separator of CallerContext. */ private final String contextFieldSeparator; /** Observer read enabled. Default for all nameservices. */ @@ -400,7 +402,7 @@ public String getAcceptedPermitsPerNsJSON() { * NN + current user. * @throws IOException If we cannot get a connection to the NameNode. */ - private ConnectionContext getConnection(UserGroupInformation ugi, String nsId, + protected ConnectionContext getConnection(UserGroupInformation ugi, String nsId, String rpcAddress, Class proto) throws IOException { ConnectionContext connection = null; try { @@ -462,7 +464,7 @@ private static IOException toIOException(Exception e) { * @return Retry decision. * @throws IOException An IO Error occurred. */ - private RetryDecision shouldRetry( + protected RetryDecision shouldRetry( final IOException ioe, final int retryCount, final String nsId, final FederationNamenodeContext namenode, final boolean listObserverFirst) throws IOException { @@ -665,7 +667,7 @@ public Object invokeMethod( * in the caller context, removing the old values if they were * already present. */ - private void addClientInfoToCallerContext(UserGroupInformation ugi) { + protected void addClientInfoToCallerContext(UserGroupInformation ugi) { CallerContext ctx = CallerContext.getCurrent(); String origContext = ctx == null ? null : ctx.getContext(); byte[] origSignature = ctx == null ? null : ctx.getSignature(); @@ -817,7 +819,7 @@ private boolean isClusterUnAvailable( * @param ioe Exception to clean up. * @return Copy of the original exception with a clean message. */ - private static IOException getCleanException(IOException ioe) { + protected static IOException getCleanException(IOException ioe) { IOException ret = null; String msg = ioe.getMessage(); @@ -1185,7 +1187,7 @@ public RemoteResult invokeSequential( * @param loc Location we are processing. * @return Exception processed for federation. */ - private IOException processException( + protected IOException processException( IOException ioe, RemoteLocationContext loc) { if (ioe instanceof RemoteException) { @@ -1251,7 +1253,7 @@ static String processExceptionMsg( * @return True if the result is an instance of the required class or if the * expected class is null. */ - private static boolean isExpectedClass(Class expectedClass, Object clazz) { + protected static boolean isExpectedClass(Class expectedClass, Object clazz) { if (expectedClass == null) { return true; } else if (clazz == null) { @@ -1269,7 +1271,7 @@ private static boolean isExpectedClass(Class expectedClass, Object clazz) { * @return True if the result is equals to the expected value or if the * expected value is null. */ - private static boolean isExpectedValue(Object expectedValue, Object value) { + protected static boolean isExpectedValue(Object expectedValue, Object value) { if (expectedValue == null) { return true; } else if (value == null) { @@ -1624,7 +1626,7 @@ public Map invokeConcurrent( * @param originContext origin CallerContext which should be transferred * to server side. */ - private void transferThreadLocalContext( + protected void transferThreadLocalContext( final Call originCall, final CallerContext originContext) { Server.getCurCall().set(originCall); CallerContext.setCurrent(originContext); @@ -1675,7 +1677,7 @@ private String getNameserviceForBlockPoolId(final String bpId) * @param controller fairness policy controller to acquire permit from * @throws IOException If permit could not be acquired for the nsId. */ - private void acquirePermit(final String nsId, final UserGroupInformation ugi, + protected void acquirePermit(final String nsId, final UserGroupInformation ugi, final RemoteMethod m, RouterRpcFairnessPolicyController controller) throws IOException { if (controller != null) { @@ -1708,7 +1710,7 @@ private void acquirePermit(final String nsId, final UserGroupInformation ugi, * @param m Remote method that needs to be invoked. * @param controller fairness policy controller to release permit from */ - private void releasePermit(final String nsId, final UserGroupInformation ugi, + protected void releasePermit(final String nsId, final UserGroupInformation ugi, final RemoteMethod m, RouterRpcFairnessPolicyController controller) { if (controller != null) { controller.releasePermit(nsId); @@ -1782,7 +1784,7 @@ private String getCurrentFairnessPolicyControllerClassName() { * @return A prioritized list of NNs to use for communication. * @throws IOException If a NN cannot be located for the nameservice ID. */ - private List getOrderedNamenodes(String nsId, + protected List getOrderedNamenodes(String nsId, boolean isObserverRead) throws IOException { final List namenodes; @@ -1802,7 +1804,7 @@ && isNamespaceStateIdFresh(nsId) return namenodes; } - private boolean isObserverReadEligible(String nsId, Method method) { + protected boolean isObserverReadEligible(String nsId, Method method) { return isReadCall(method) && isNamespaceObserverReadEligible(nsId); } @@ -1857,7 +1859,7 @@ private LongAccumulator getTimeOfLastCallToActive(String namespaceId) { * {@link RouterRpcClient#isUnavailableException(IOException) unavailable exception}, * otherwise false. */ - private boolean shouldRotateCache(IOException ioe) { + protected boolean shouldRotateCache(IOException ioe) { if (isUnavailableException(ioe)) { return true; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java index 27385a2686ce5..767e8c978b692 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java @@ -68,6 +68,9 @@ void init( */ void proxyOpComplete(boolean success, String nsId, FederationNamenodeServiceState state); + void proxyOpComplete( + boolean success, String nsId, FederationNamenodeServiceState state, long costMs); + /** * Failed to proxy an operation to a namenode because it was in standby. * @param nsId nameservice id. @@ -138,4 +141,6 @@ void init( * If a path is in a read only mount point. */ void routerFailureReadOnly(); + + void incrProcessingOp(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java index 217c62ff28762..a353e20b8f0c6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java @@ -30,6 +30,8 @@ import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DN_REPORT_CACHE_EXPIRE_MS_DEFAULT; import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION; import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_FEDERATION_RENAME_OPTION_DEFAULT; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.getResult; import static org.apache.hadoop.hdfs.server.federation.router.RouterFederationRename.RouterRenameOption; import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI; @@ -49,15 +51,27 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.protocolPB.RouterClientNamenodeProtocolServerSideTranslatorPB; +import org.apache.hadoop.hdfs.protocolPB.RouterGetUserMappingsProtocolServerSideTranslatorPB; +import org.apache.hadoop.hdfs.protocolPB.RouterNamenodeProtocolServerSideTranslatorPB; +import org.apache.hadoop.hdfs.protocolPB.RouterRefreshUserMappingsProtocolServerSideTranslatorPB; +import org.apache.hadoop.hdfs.protocolPB.RouterClientProtocolTranslatorPB; import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder; import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader; import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache; @@ -185,6 +199,9 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.thirdparty.protobuf.BlockingService; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.getCompletableFuture; +import static org.apache.hadoop.hdfs.server.federation.router.RouterAsyncRpcUtil.setCurCompletableFuture; + /** * This class is responsible for handling all of the RPC calls to the It is * created, started, and stopped by {@link Router}. It implements the @@ -198,7 +215,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, private static final Logger LOG = LoggerFactory.getLogger(RouterRpcServer.class); - + private static volatile ExecutorService executor; /** Configuration for the RPC server. */ private Configuration conf; @@ -255,6 +272,19 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, private RouterRenameOption routerRenameOption; /** Schedule the router federation rename jobs. */ private BalanceProcedureScheduler fedRenameScheduler; + + public static Executor getExecutor() { + if (executor == null) { + synchronized (RouterRpcServer.class) { + if (executor == null) { + executor = Executors.newFixedThreadPool(100); + } + return executor; + } + } + return executor; + } + /** * Construct a router RPC server. * @@ -268,7 +298,7 @@ public RouterRpcServer(Configuration conf, Router router, ActiveNamenodeResolver nnResolver, FileSubclusterResolver fileResolver) throws IOException { super(RouterRpcServer.class.getName()); - + executor = null; this.conf = conf; this.router = router; this.namenodeResolver = nnResolver; @@ -294,25 +324,30 @@ public RouterRpcServer(Configuration conf, Router router, RPC.setProtocolEngine(this.conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine2.class); - ClientNamenodeProtocolServerSideTranslatorPB - clientProtocolServerTranslator = - new ClientNamenodeProtocolServerSideTranslatorPB(this); + ClientNamenodeProtocolServerSideTranslatorPB clientProtocolServerTranslator; + if (isAsync()) { + clientProtocolServerTranslator = + new RouterClientNamenodeProtocolServerSideTranslatorPB(this); + } else { + clientProtocolServerTranslator = + new ClientNamenodeProtocolServerSideTranslatorPB(this); + } BlockingService clientNNPbService = ClientNamenodeProtocol .newReflectiveBlockingService(clientProtocolServerTranslator); NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = - new NamenodeProtocolServerSideTranslatorPB(this); + new RouterNamenodeProtocolServerSideTranslatorPB(this); BlockingService nnPbService = NamenodeProtocolService .newReflectiveBlockingService(namenodeProtocolXlator); RefreshUserMappingsProtocolServerSideTranslatorPB refreshUserMappingXlator = - new RefreshUserMappingsProtocolServerSideTranslatorPB(this); + new RouterRefreshUserMappingsProtocolServerSideTranslatorPB(this); BlockingService refreshUserMappingService = RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService. newReflectiveBlockingService(refreshUserMappingXlator); GetUserMappingsProtocolServerSideTranslatorPB getUserMappingXlator = - new GetUserMappingsProtocolServerSideTranslatorPB(this); + new RouterGetUserMappingsProtocolServerSideTranslatorPB(this); BlockingService getUserMappingService = GetUserMappingsProtocolProtos.GetUserMappingsProtocolService. newReflectiveBlockingService(getUserMappingXlator); @@ -391,14 +426,21 @@ public RouterRpcServer(Configuration conf, Router router, } // Create the client - this.rpcClient = new RouterRpcClient(this.conf, this.router, - this.namenodeResolver, this.rpcMonitor, routerStateIdContext); - - // Initialize modules - this.quotaCall = new Quota(this.router, this); - this.nnProto = new RouterNamenodeProtocol(this); - this.clientProto = new RouterClientProtocol(conf, this); - this.routerProto = new RouterUserProtocol(this); + if (router.isEnableAsync()) { + this.rpcClient = new RouterAsyncRpcClient(this.conf, this.router, + this.namenodeResolver, this.rpcMonitor, routerStateIdContext); + this.nnProto = new RouterAsyncNamenodeProtocol(this); + this.quotaCall = new AsyncQuota(this.router, this); + this.clientProto = new RouterAsyncClientProtocol(conf, this); + this.routerProto = new RouterAsyncUserProtocol(this); + } else { + this.rpcClient = new RouterRpcClient(this.conf, this.router, + this.namenodeResolver, this.rpcMonitor, routerStateIdContext); + this.nnProto = new RouterNamenodeProtocol(this); + this.quotaCall = new Quota(this.router, this); + this.clientProto = new RouterClientProtocol(conf, this); + this.routerProto = new RouterUserProtocol(this); + } long dnCacheExpire = conf.getTimeDuration( DN_REPORT_CACHE_EXPIRE, @@ -537,6 +579,9 @@ protected void serviceStop() throws Exception { if (this.fedRenameScheduler != null) { fedRenameScheduler.shutDown(); } + if (executor != null) { + executor.shutdown(); + } super.serviceStop(); } @@ -782,6 +827,116 @@ T invokeOnNs(RemoteMethod method, Class clazz, IOException ioe, throw ioe; } + T invokeAtAvailableNsAsync(RemoteMethod method, Class clazz) + throws IOException { + String nsId = subclusterResolver.getDefaultNamespace(); + Set nss = namenodeResolver.getNamespaces(); + // If no namespace is available, then throw this IOException. + IOException io = new IOException("No namespace available."); + + CompletableFuture completableFuture = + CompletableFuture.completedFuture(new Object[]{null, false}); + // If default Ns is present return result from that namespace. + if (!nsId.isEmpty()) { + completableFuture = completableFuture.thenCompose(o -> { + try { + rpcClient.invokeSingle(nsId, method, clazz); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + }).handle((o, e) -> { + if (e == null) { + return new Object[]{o, true}; + } + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + IOException ioe = (IOException) cause; + if (!clientProto.isUnavailableSubclusterException(ioe)) { + LOG.debug("{} exception cannot be retried", + ioe.getClass().getSimpleName()); + throw new CompletionException(ioe); + } + return new Object[]{null, false}; + } + throw new CompletionException(cause); + }).thenCompose(o -> { + Object[] args = o; + boolean complete = (boolean) args[1]; + if (complete) { + return CompletableFuture.completedFuture(args[0]); + } + // Remove the already tried namespace. + nss.removeIf(n -> n.getNameserviceId().equals(nsId)); + try { + invokeOnNsAsync(method, clazz, io, nss); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + }); + } else { + invokeOnNsAsync(method, clazz, io, nss); + completableFuture = getCompletableFuture(); + } + setCurCompletableFuture(completableFuture); + return asyncReturn(clazz); + } + + T invokeOnNsAsync(RemoteMethod method, Class clazz, IOException ioe, + Set nss) throws IOException { + if (nss.isEmpty()) { + throw ioe; + } + CompletableFuture completableFuture = + CompletableFuture.completedFuture(new Object[] {null, false}); + for (FederationNamespaceInfo fnInfo : nss) { + String nsId = fnInfo.getNameserviceId(); + LOG.debug("Invoking {} on namespace {}", method, nsId); + completableFuture = completableFuture.thenCompose(o -> { + Object[] args = (Object[]) o; + boolean complete = (boolean) args[1]; + if (complete) { + return CompletableFuture.completedFuture(args[0]); + } + try { + rpcClient.invokeSingle(nsId, method, clazz); + return getCompletableFuture(); + } catch (IOException e) { + throw new CompletionException(e); + } + }).handle((o, e) -> { + if (e == null) { + return new Object[]{o, true}; + } + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + IOException ioe1 = (IOException) cause; + LOG.debug("Failed to invoke {} on namespace {}", method, nsId, ioe1); + // Ignore the exception and try on other namespace, if the tried + // namespace is unavailable, else throw the received exception. + if (!clientProto.isUnavailableSubclusterException(ioe1)) { + throw new CompletionException(ioe1); + } + return new Object[] {null, false}; + } + throw new CompletionException(cause); + }); + } + + completableFuture = completableFuture.thenApply(o -> { + Object[] args = (Object[]) o; + boolean complete = (boolean) args[1]; + if (!complete) { + // Couldn't get a response from any of the namespace, throw ioe. + throw new CompletionException(ioe); + } + return args[0]; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(clazz); + } + @Override // ClientProtocol public Token getDelegationToken(Text renewer) throws IOException { @@ -833,6 +988,10 @@ public HdfsFileStatus create(String src, FsPermission masked, */ RemoteLocation getCreateLocation(final String src) throws IOException { final List locations = getLocationsForPath(src, true); + if (router.isEnableAsync()) { + getCreateLocationAsync(src, locations); + return asyncReturn(RemoteLocation.class); + } return getCreateLocation(src, locations); } @@ -869,6 +1028,47 @@ RemoteLocation getCreateLocation( return createLocation; } + RemoteLocation getCreateLocationAsync( + final String src, final List locations) + throws IOException { + + if (locations == null || locations.isEmpty()) { + throw new IOException("Cannot get locations to create " + src); + } + + RemoteLocation createLocation = locations.get(0); + CompletableFuture completableFuture = + CompletableFuture.completedFuture(createLocation); + if (locations.size() > 1) { + completableFuture = completableFuture.thenCompose(o -> { + try { + getExistingLocationAsync(src, locations); + return getCompletableFuture().thenApply(existingLocation -> { + if (existingLocation != null) { + LOG.debug("{} already exists in {}.", src, existingLocation); + return existingLocation; + } + return createLocation; + }); + } catch (IOException e) { + throw new CompletionException(e); + } + }).handle((o, e) -> { + if (e == null) { + return o; + } + Throwable cause = e.getCause(); + if (cause instanceof FileNotFoundException) { + return o; + } + throw new CompletionException(cause); + }); + } + setCurCompletableFuture(completableFuture); + return asyncReturn(RemoteLocation.class); + } + + /** * Gets the remote location where the file exists. * @param src the name of file. @@ -890,6 +1090,27 @@ private RemoteLocation getExistingLocation(String src, return null; } + private RemoteLocation getExistingLocationAsync( + String src, List locations) throws IOException { + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + rpcClient.invokeConcurrent( + locations, method, true, false, HdfsFileStatus.class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results = + (Map) o; + for (RemoteLocation loc : locations) { + if (results.get(loc) != null) { + return loc; + } + } + return null; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(RemoteLocation.class); + } + @Override // ClientProtocol public LastBlockWithStatus append(String src, final String clientName, final EnumSetWritable flag) throws IOException { @@ -1109,6 +1330,9 @@ private DatanodeInfo[] getCachedDatanodeReportImpl( try { DatanodeInfo[] dns = clientProto.getDatanodeReport(type); + if (isAsync()) { + dns = (DatanodeInfo[]) getResult(); + } LOG.debug("Refresh cached DN report with {} datanodes", dns.length); return dns; } finally { @@ -1144,6 +1368,34 @@ public DatanodeInfo[] getDatanodeReport( return toArray(datanodes, DatanodeInfo.class); } + public DatanodeInfo[] getDatanodeReportAsync( + DatanodeReportType type, boolean requireResponse, long timeOutMs) + throws IOException { + checkOperation(OperationCategory.UNCHECKED); + + Map datanodesMap = new LinkedHashMap<>(); + RemoteMethod method = new RemoteMethod("getDatanodeReport", + new Class[] {DatanodeReportType.class}, type); + + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, requireResponse, false, + timeOutMs, DatanodeInfo[].class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(new Function() { + @Override + public Object apply(Object o) { + Map results = + (Map) o; + updateDnMap(results, datanodesMap); + // Map -> Array + Collection datanodes = datanodesMap.values(); + return toArray(datanodes, DatanodeInfo.class); + } + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(DatanodeInfo[].class); + } + @Override // ClientProtocol public DatanodeStorageReport[] getDatanodeStorageReport( DatanodeReportType type) throws IOException { @@ -1162,6 +1414,11 @@ public Map getDatanodeStorageReportMap( return getDatanodeStorageReportMap(type, true, -1); } + public Map getDatanodeStorageReportMapAsync( + DatanodeReportType type) throws IOException { + return getDatanodeStorageReportMapAsync(type, true, -1); + } + /** * Get the list of datanodes per subcluster. * @@ -1194,6 +1451,33 @@ public Map getDatanodeStorageReportMap( return ret; } + public Map getDatanodeStorageReportMapAsync( + DatanodeReportType type, boolean requireResponse, long timeOutMs) + throws IOException { + + RemoteMethod method = new RemoteMethod("getDatanodeStorageReport", + new Class[] {DatanodeReportType.class}, type); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent( + nss, method, requireResponse, false, timeOutMs, DatanodeStorageReport[].class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results = + (Map) o; + Map ret = new LinkedHashMap<>(); + for (Entry entry : + results.entrySet()) { + FederationNamespaceInfo ns = entry.getKey(); + String nsId = ns.getNameserviceId(); + DatanodeStorageReport[] result = entry.getValue(); + ret.put(nsId, result); + } + return ret; + }); + setCurCompletableFuture(completableFuture); + return asyncReturn(Map.class); + } + @Override // ClientProtocol public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException { @@ -2009,6 +2293,29 @@ public DatanodeInfo[] getSlowDatanodeReport(boolean requireResponse, long timeOu return toArray(datanodes, DatanodeInfo.class); } + public DatanodeInfo[] getSlowDatanodeReportAsync( + boolean requireResponse, long timeOutMs) throws IOException { + checkOperation(OperationCategory.UNCHECKED); + + Map datanodesMap = new LinkedHashMap<>(); + RemoteMethod method = new RemoteMethod("getSlowDatanodeReport"); + + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, requireResponse, false, + timeOutMs, DatanodeInfo[].class); + CompletableFuture completableFuture = getCompletableFuture(); + completableFuture = completableFuture.thenApply(o -> { + Map results = + (Map) o; + updateDnMap(results, datanodesMap); + // Map -> Array + Collection datanodes = datanodesMap.values(); + return toArray(datanodes, DatanodeInfo.class); + }); + setCurCompletableFuture(completableFuture); + return null; + } + private void updateDnMap(Map results, Map datanodesMap) { for (Entry entry : @@ -2066,4 +2373,8 @@ public ListenableFuture reload( return executorService.submit(() -> load(type)); } } + + public boolean isAsync() { + return router.isEnableAsync(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSnapshot.java index 9a47f2a012a0a..eecfadeec5e7b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSnapshot.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterSnapshot.java @@ -244,4 +244,16 @@ public SnapshotDiffReportListing getSnapshotDiffReportListing( locations, remoteMethod, SnapshotDiffReportListing.class, null); } } + + public RouterRpcServer getRpcServer() { + return rpcServer; + } + + public RouterRpcClient getRpcClient() { + return rpcClient; + } + + public ActiveNamenodeResolver getNamenodeResolver() { + return namenodeResolver; + } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStoragePolicy.java index 05f983c397297..433cdad72bda6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStoragePolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStoragePolicy.java @@ -89,7 +89,7 @@ public BlockStoragePolicy getStoragePolicy(String path) RemoteMethod method = new RemoteMethod("getStoragePolicy", new Class[] {String.class}, new RemoteParam()); - return (BlockStoragePolicy) rpcClient.invokeSequential(locations, method); + return rpcClient.invokeSequential(locations, method); } public void satisfyStoragePolicy(String path) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java index cbc11b27b2b05..f571ab3277703 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.io.FileNotFoundException; import java.io.IOException; @@ -50,6 +51,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.SnapshotStatus; +import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.RouterContext; import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster; @@ -490,19 +492,32 @@ public void testIsMultiDestDir() throws Exception { routerContext.getRouter().getRpcServer().getClientProtocolModule(); setupOrderMountPath(DestinationOrder.HASH_ALL); // Should be true only for directory and false for all other cases. - assertTrue(client.isMultiDestDirectory("/mount/dir")); - assertFalse(client.isMultiDestDirectory("/mount/nodir")); - assertFalse(client.isMultiDestDirectory("/mount/dir/file")); + assertTrue(isMultiDestDir(client, "/mount/dir")); + assertFalse(isMultiDestDir(client, "/mount/nodir")); + assertFalse(isMultiDestDir(client, "/mount/dir/file")); routerFs.createSymlink(new Path("/mount/dir/file"), new Path("/mount/dir/link"), true); - assertFalse(client.isMultiDestDirectory("/mount/dir/link")); + assertFalse(isMultiDestDir(client, "/mount/dir/link")); routerFs.createSymlink(new Path("/mount/dir/dir"), new Path("/mount/dir/linkDir"), true); - assertFalse(client.isMultiDestDirectory("/mount/dir/linkDir")); + assertFalse(isMultiDestDir(client, "/mount/dir/linkDir")); resetTestEnvironment(); // Test single directory destination. Should be false for the directory. setupOrderMountPath(DestinationOrder.HASH); - assertFalse(client.isMultiDestDirectory("/mount/dir")); + assertFalse(isMultiDestDir(client, "/mount/dir")); + } + + private boolean isMultiDestDir( + RouterClientProtocol client, String src) throws IOException { + boolean multiDestDirectory = client.isMultiDestDirectory(src); + if (rpcServer.isAsync()) { + try { + multiDestDirectory = (boolean) RouterAsyncRpcUtil.getResult(); + } catch (UnresolvedPathException e) { + return false; + } + } + return multiDestDirectory; } /** @@ -675,8 +690,14 @@ public void testInvokeAtAvailableNs() throws IOException { try { // Verify that #invokeAtAvailableNs works by calling #getServerDefaults. RemoteMethod method = new RemoteMethod("getServerDefaults"); - FsServerDefaults serverDefaults = - rpcServer.invokeAtAvailableNs(method, FsServerDefaults.class); + FsServerDefaults serverDefaults = null; + if (rpcServer.isAsync()) { + rpcServer.invokeAtAvailableNsAsync(method, FsServerDefaults.class); + serverDefaults = (FsServerDefaults) RouterAsyncRpcUtil.getResult(); + } else { + serverDefaults = + rpcServer.invokeAtAvailableNs(method, FsServerDefaults.class); + } assertNotNull(serverDefaults); } finally { dfsCluster.restartNameNode(0); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java index c84dd2ceb2060..365a6760b29cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java @@ -2011,6 +2011,9 @@ public void testgetGroupsForUser() throws IOException { new String[] {"bar", "group2"}); String[] result = router.getRouter().getRpcServer().getGroupsForUser("user"); + if (router.getRouter().isEnableAsync()) { + result = (String[]) RouterAsyncRpcUtil.getResult(); + } assertArrayEquals(group, result); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/testcom.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/testcom.java new file mode 100644 index 0000000000000..20970a7e0057d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/testcom.java @@ -0,0 +1,71 @@ +package org.apache.hadoop; + +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; +import java.util.function.Supplier; + + +@SuppressWarnings("checkstyle:TypeName") +public class testcom { + + @Test + public void test1() throws InterruptedException, ExecutionException { + CompletableFuture future = CompletableFuture.supplyAsync(() -> { + int i = 0; + if (i == 1) { + return "zj"; + } + throw new RuntimeException(new IOException("asdasdsadsadasd")); + // 可能抛出异常的代码 + }).thenApplyAsync(res -> { + // 使用结果的代码 + System.out.println("[adasd]"); + return res; + }); + //.exceptionally(ex -> { +// // 处理异常 +// throw new CompletionException(ex); +// }).thenApplyAsync(s -> { +// System.out.println("[zz]"+s); +// +// return s; +// }).thenCompose(s -> CompletableFuture.supplyAsync(() -> "[asdasdasdsadsa]")) +// .exceptionally(throwable -> { +// System.out.println(throwable); +// throwable.printStackTrace(); +// return "aa"; +// }); + + + System.out.println(future.get()); + } + + @Test + public void test2() throws ExecutionException, InterruptedException { + CompletableFuture completableFuture = CompletableFuture.completedFuture("0"); + List list = new ArrayList<>(); + list.add("1"); + list.add("2"); + list.add("3"); + + for (String s : list) { + completableFuture = completableFuture.thenCompose(o -> { + System.out.println(s); + if (s.equals("2")) { + throw new CompletionException(new IOException("asdasdasd")); + } + return CompletableFuture.completedFuture(o + s); + }); + } + + System.out.println(completableFuture.get()); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java index 25fcdc3080df0..7f5140139fb01 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java @@ -321,131 +321,131 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements ClientNamenodeProtocolPB { final private ClientProtocol server; - static final DeleteSnapshotResponseProto VOID_DELETE_SNAPSHOT_RESPONSE = + protected static final DeleteSnapshotResponseProto VOID_DELETE_SNAPSHOT_RESPONSE = DeleteSnapshotResponseProto.newBuilder().build(); - static final RenameSnapshotResponseProto VOID_RENAME_SNAPSHOT_RESPONSE = + protected static final RenameSnapshotResponseProto VOID_RENAME_SNAPSHOT_RESPONSE = RenameSnapshotResponseProto.newBuilder().build(); - static final AllowSnapshotResponseProto VOID_ALLOW_SNAPSHOT_RESPONSE = + protected static final AllowSnapshotResponseProto VOID_ALLOW_SNAPSHOT_RESPONSE = AllowSnapshotResponseProto.newBuilder().build(); - static final DisallowSnapshotResponseProto VOID_DISALLOW_SNAPSHOT_RESPONSE = + protected static final DisallowSnapshotResponseProto VOID_DISALLOW_SNAPSHOT_RESPONSE = DisallowSnapshotResponseProto.newBuilder().build(); - static final GetSnapshottableDirListingResponseProto + protected static final GetSnapshottableDirListingResponseProto NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE = GetSnapshottableDirListingResponseProto.newBuilder().build(); - static final GetSnapshotListingResponseProto + protected static final GetSnapshotListingResponseProto NULL_GET_SNAPSHOT_LISTING_RESPONSE = GetSnapshotListingResponseProto.newBuilder().build(); - static final SetStoragePolicyResponseProto VOID_SET_STORAGE_POLICY_RESPONSE = + protected static final SetStoragePolicyResponseProto VOID_SET_STORAGE_POLICY_RESPONSE = SetStoragePolicyResponseProto.newBuilder().build(); - static final UnsetStoragePolicyResponseProto + protected static final UnsetStoragePolicyResponseProto VOID_UNSET_STORAGE_POLICY_RESPONSE = UnsetStoragePolicyResponseProto.newBuilder().build(); - private static final CreateResponseProto VOID_CREATE_RESPONSE = + protected static final CreateResponseProto VOID_CREATE_RESPONSE = CreateResponseProto.newBuilder().build(); - private static final SetPermissionResponseProto VOID_SET_PERM_RESPONSE = + protected static final SetPermissionResponseProto VOID_SET_PERM_RESPONSE = SetPermissionResponseProto.newBuilder().build(); - private static final SetOwnerResponseProto VOID_SET_OWNER_RESPONSE = + protected static final SetOwnerResponseProto VOID_SET_OWNER_RESPONSE = SetOwnerResponseProto.newBuilder().build(); - private static final AbandonBlockResponseProto VOID_ADD_BLOCK_RESPONSE = + protected static final AbandonBlockResponseProto VOID_ADD_BLOCK_RESPONSE = AbandonBlockResponseProto.newBuilder().build(); - private static final ReportBadBlocksResponseProto VOID_REP_BAD_BLOCK_RESPONSE = + protected static final ReportBadBlocksResponseProto VOID_REP_BAD_BLOCK_RESPONSE = ReportBadBlocksResponseProto.newBuilder().build(); - private static final ConcatResponseProto VOID_CONCAT_RESPONSE = + protected static final ConcatResponseProto VOID_CONCAT_RESPONSE = ConcatResponseProto.newBuilder().build(); - private static final Rename2ResponseProto VOID_RENAME2_RESPONSE = + protected static final Rename2ResponseProto VOID_RENAME2_RESPONSE = Rename2ResponseProto.newBuilder().build(); - private static final GetListingResponseProto VOID_GETLISTING_RESPONSE = + protected static final GetListingResponseProto VOID_GETLISTING_RESPONSE = GetListingResponseProto.newBuilder().build(); - private static final GetBatchedListingResponseProto + protected static final GetBatchedListingResponseProto VOID_GETBATCHEDLISTING_RESPONSE = GetBatchedListingResponseProto.newBuilder() .setStartAfter(ByteString.copyFromUtf8("")) .setHasMore(false) .build(); - private static final RenewLeaseResponseProto VOID_RENEWLEASE_RESPONSE = + protected static final RenewLeaseResponseProto VOID_RENEWLEASE_RESPONSE = RenewLeaseResponseProto.newBuilder().build(); - private static final RefreshNodesResponseProto VOID_REFRESHNODES_RESPONSE = + protected static final RefreshNodesResponseProto VOID_REFRESHNODES_RESPONSE = RefreshNodesResponseProto.newBuilder().build(); - private static final FinalizeUpgradeResponseProto VOID_FINALIZEUPGRADE_RESPONSE = + protected static final FinalizeUpgradeResponseProto VOID_FINALIZEUPGRADE_RESPONSE = FinalizeUpgradeResponseProto.newBuilder().build(); - private static final MetaSaveResponseProto VOID_METASAVE_RESPONSE = + protected static final MetaSaveResponseProto VOID_METASAVE_RESPONSE = MetaSaveResponseProto.newBuilder().build(); - private static final GetFileInfoResponseProto VOID_GETFILEINFO_RESPONSE = + protected static final GetFileInfoResponseProto VOID_GETFILEINFO_RESPONSE = GetFileInfoResponseProto.newBuilder().build(); - private static final GetLocatedFileInfoResponseProto + protected static final GetLocatedFileInfoResponseProto VOID_GETLOCATEDFILEINFO_RESPONSE = GetLocatedFileInfoResponseProto.newBuilder().build(); - private static final GetFileLinkInfoResponseProto VOID_GETFILELINKINFO_RESPONSE = + protected static final GetFileLinkInfoResponseProto VOID_GETFILELINKINFO_RESPONSE = GetFileLinkInfoResponseProto.newBuilder().build(); - private static final SetQuotaResponseProto VOID_SETQUOTA_RESPONSE = + protected static final SetQuotaResponseProto VOID_SETQUOTA_RESPONSE = SetQuotaResponseProto.newBuilder().build(); - private static final FsyncResponseProto VOID_FSYNC_RESPONSE = + protected static final FsyncResponseProto VOID_FSYNC_RESPONSE = FsyncResponseProto.newBuilder().build(); - private static final SetTimesResponseProto VOID_SETTIMES_RESPONSE = + protected static final SetTimesResponseProto VOID_SETTIMES_RESPONSE = SetTimesResponseProto.newBuilder().build(); - private static final CreateSymlinkResponseProto VOID_CREATESYMLINK_RESPONSE = + protected static final CreateSymlinkResponseProto VOID_CREATESYMLINK_RESPONSE = CreateSymlinkResponseProto.newBuilder().build(); - private static final UpdatePipelineResponseProto + protected static final UpdatePipelineResponseProto VOID_UPDATEPIPELINE_RESPONSE = UpdatePipelineResponseProto.newBuilder().build(); - private static final CancelDelegationTokenResponseProto + protected static final CancelDelegationTokenResponseProto VOID_CANCELDELEGATIONTOKEN_RESPONSE = CancelDelegationTokenResponseProto.newBuilder().build(); - private static final SetBalancerBandwidthResponseProto + protected static final SetBalancerBandwidthResponseProto VOID_SETBALANCERBANDWIDTH_RESPONSE = SetBalancerBandwidthResponseProto.newBuilder().build(); - private static final SetAclResponseProto + protected static final SetAclResponseProto VOID_SETACL_RESPONSE = SetAclResponseProto.getDefaultInstance(); - private static final ModifyAclEntriesResponseProto + protected static final ModifyAclEntriesResponseProto VOID_MODIFYACLENTRIES_RESPONSE = ModifyAclEntriesResponseProto .getDefaultInstance(); - private static final RemoveAclEntriesResponseProto + protected static final RemoveAclEntriesResponseProto VOID_REMOVEACLENTRIES_RESPONSE = RemoveAclEntriesResponseProto .getDefaultInstance(); - private static final RemoveDefaultAclResponseProto + protected static final RemoveDefaultAclResponseProto VOID_REMOVEDEFAULTACL_RESPONSE = RemoveDefaultAclResponseProto .getDefaultInstance(); - private static final RemoveAclResponseProto + protected static final RemoveAclResponseProto VOID_REMOVEACL_RESPONSE = RemoveAclResponseProto.getDefaultInstance(); - - private static final SetXAttrResponseProto + + protected static final SetXAttrResponseProto VOID_SETXATTR_RESPONSE = SetXAttrResponseProto.getDefaultInstance(); - - private static final RemoveXAttrResponseProto + + protected static final RemoveXAttrResponseProto VOID_REMOVEXATTR_RESPONSE = RemoveXAttrResponseProto.getDefaultInstance(); - private static final CheckAccessResponseProto + protected static final CheckAccessResponseProto VOID_CHECKACCESS_RESPONSE = CheckAccessResponseProto.getDefaultInstance(); - private static final SatisfyStoragePolicyResponseProto + protected static final SatisfyStoragePolicyResponseProto VOID_SATISFYSTORAGEPOLICY_RESPONSE = SatisfyStoragePolicyResponseProto .getDefaultInstance(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java index da2440dfa722a..251064654cc37 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolServerSideTranslatorPB.java @@ -73,11 +73,11 @@ public class NamenodeProtocolServerSideTranslatorPB implements NamenodeProtocolPB { private final NamenodeProtocol impl; - private final static ErrorReportResponseProto VOID_ERROR_REPORT_RESPONSE = - ErrorReportResponseProto.newBuilder().build(); + protected final static ErrorReportResponseProto VOID_ERROR_REPORT_RESPONSE = + ErrorReportResponseProto.newBuilder().build(); - private final static EndCheckpointResponseProto VOID_END_CHECKPOINT_RESPONSE = - EndCheckpointResponseProto.newBuilder().build(); + protected final static EndCheckpointResponseProto VOID_END_CHECKPOINT_RESPONSE = + EndCheckpointResponseProto.newBuilder().build(); public NamenodeProtocolServerSideTranslatorPB(NamenodeProtocol impl) { this.impl = impl; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java index 40d0395721be4..a1b40e4c7d9c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java @@ -72,18 +72,18 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol, ProtocolMetaInterface, Closeable, ProtocolTranslator { /** RpcController is not used and hence is set to null */ - private final static RpcController NULL_CONTROLLER = null; + protected final static RpcController NULL_CONTROLLER = null; /* * Protobuf requests with no parameters instantiated only once */ - private static final GetBlockKeysRequestProto VOID_GET_BLOCKKEYS_REQUEST = + protected static final GetBlockKeysRequestProto VOID_GET_BLOCKKEYS_REQUEST = GetBlockKeysRequestProto.newBuilder().build(); - private static final GetTransactionIdRequestProto VOID_GET_TRANSACTIONID_REQUEST = + protected static final GetTransactionIdRequestProto VOID_GET_TRANSACTIONID_REQUEST = GetTransactionIdRequestProto.newBuilder().build(); - private static final RollEditLogRequestProto VOID_ROLL_EDITLOG_REQUEST = + protected static final RollEditLogRequestProto VOID_ROLL_EDITLOG_REQUEST = RollEditLogRequestProto.newBuilder().build(); - private static final VersionRequestProto VOID_VERSION_REQUEST = + protected static final VersionRequestProto VOID_VERSION_REQUEST = VersionRequestProto.newBuilder().build(); final private NamenodeProtocolPB rpcProxy;