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..f052aa2d8a260 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,8 +96,8 @@ public class Client implements AutoCloseable { private static final ThreadLocal retryCount = new ThreadLocal(); private static final ThreadLocal EXTERNAL_CALL_HANDLER = new ThreadLocal<>(); - private static final ThreadLocal> - ASYNC_RPC_RESPONSE = new ThreadLocal<>(); + private static final ThreadLocal> ASYNC_RPC_RESPONSE + = new ThreadLocal<>(); private static final ThreadLocal asynchronousMode = new ThreadLocal() { @Override @@ -110,7 +110,46 @@ protected Boolean initialValue() { @Unstable public static AsyncGet getAsyncRpcResponse() { - return (AsyncGet) ASYNC_RPC_RESPONSE.get(); + CompletableFuture responseFuture = ASYNC_RPC_RESPONSE.get(); + return new AsyncGet() { + @Override + public T get(long timeout, TimeUnit unit) + throws IOException, TimeoutException, InterruptedException { + try { + if (unit == null || timeout < 0) { + return (T) responseFuture.get(); + } + return (T) responseFuture.get(timeout, unit); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + throw (IOException) cause; + } + throw new IllegalStateException(e); + } + } + + @Override + public boolean isDone() { + return responseFuture.isDone(); + } + }; + } + + /** + * Retrieves the current response future from the thread-local storage. + * + * @return A {@link CompletableFuture} of type T that represents the + * asynchronous operation. If no response future is present in + * the thread-local storage, this method returns {@code null}. + * @param The type of the value completed by the returned + * {@link CompletableFuture}. It must be a subclass of + * {@link Writable}. + * @see CompletableFuture + * @see Writable + */ + public static CompletableFuture getResponseFuture() { + return (CompletableFuture) ASYNC_RPC_RESPONSE.get(); } /** @@ -277,10 +316,8 @@ static class Call { final int id; // call id final int retry; // retry count final Writable rpcRequest; // the serialized rpc request - Writable rpcResponse; // null if rpc has error - IOException error; // exception, null if success + private final CompletableFuture rpcResponseFuture; final RPC.RpcKind rpcKind; // Rpc EngineKind - boolean done; // true when call is done private final Object externalHandler; private AlignmentContext alignmentContext; @@ -304,6 +341,7 @@ private Call(RPC.RpcKind rpcKind, Writable param) { } this.externalHandler = EXTERNAL_CALL_HANDLER.get(); + this.rpcResponseFuture = new CompletableFuture<>(); } @Override @@ -314,9 +352,6 @@ public String toString() { /** Indicate when the call is complete and the * value or error are available. Notifies by default. */ protected synchronized void callComplete() { - this.done = true; - notify(); // notify caller - if (externalHandler != null) { synchronized (externalHandler) { externalHandler.notify(); @@ -339,7 +374,7 @@ public synchronized void setAlignmentContext(AlignmentContext ac) { * @param error exception thrown by the call; either local or remote */ public synchronized void setException(IOException error) { - this.error = error; + rpcResponseFuture.completeExceptionally(error); callComplete(); } @@ -349,13 +384,9 @@ public synchronized void setException(IOException error) { * @param rpcResponse return value of the rpc call. */ public synchronized void setRpcResponse(Writable rpcResponse) { - this.rpcResponse = rpcResponse; + rpcResponseFuture.complete(rpcResponse); callComplete(); } - - public synchronized Writable getRpcResponse() { - return rpcResponse; - } } /** Thread that reads responses and notifies callers. Each connection owns a @@ -1495,39 +1526,19 @@ Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, } if (isAsynchronousMode()) { - final AsyncGet asyncGet - = new AsyncGet() { - @Override - public Writable get(long timeout, TimeUnit unit) - throws IOException, TimeoutException{ - boolean done = true; - try { - final Writable w = getRpcResponse(call, connection, timeout, unit); - if (w == null) { - done = false; - throw new TimeoutException(call + " timed out " - + timeout + " " + unit); - } - return w; - } finally { - if (done) { - releaseAsyncCall(); + CompletableFuture result = call.rpcResponseFuture.handle( + (rpcResponse, e) -> { + releaseAsyncCall(); + if (e != null) { + IOException ioe = (IOException) e; + throw new CompletionException(warpIOException(ioe, connection)); } - } - } - - @Override - public boolean isDone() { - synchronized (call) { - return call.done; - } - } - }; - - ASYNC_RPC_RESPONSE.set(asyncGet); + return rpcResponse; + }); + ASYNC_RPC_RESPONSE.set(result); return null; } else { - return getRpcResponse(call, connection, -1, null); + return getRpcResponse(call, connection); } } @@ -1564,37 +1575,34 @@ int getAsyncCallCount() { } /** @return the rpc response or, in case of timeout, null. */ - private Writable getRpcResponse(final Call call, final Connection connection, - final long timeout, final TimeUnit unit) throws IOException { - synchronized (call) { - while (!call.done) { - try { - AsyncGet.Util.wait(call, timeout, unit); - if (timeout >= 0 && !call.done) { - return null; - } - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - throw new InterruptedIOException("Call interrupted"); - } + private Writable getRpcResponse(final Call call, final Connection connection) + throws IOException { + try { + return call.rpcResponseFuture.get(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new InterruptedIOException("Call interrupted"); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + throw warpIOException((IOException) cause, connection); } + throw new IllegalStateException(e); + } + } - if (call.error != null) { - if (call.error instanceof RemoteException || - call.error instanceof SaslException) { - call.error.fillInStackTrace(); - throw call.error; - } else { // local exception - InetSocketAddress address = connection.getRemoteAddress(); - throw NetUtils.wrapException(address.getHostName(), - address.getPort(), - NetUtils.getHostname(), - 0, - call.error); - } - } else { - return call.getRpcResponse(); - } + private IOException warpIOException(IOException ioe, Connection connection) { + if (ioe instanceof RemoteException || + ioe instanceof SaslException) { + ioe.fillInStackTrace(); + return ioe; + } else { // local exception + InetSocketAddress address = connection.getRemoteAddress(); + return NetUtils.wrapException(address.getHostName(), + address.getPort(), + NetUtils.getHostname(), + 0, + ioe); } } 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 239437854dda5..3a1e732cd9858 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 @@ -214,7 +214,7 @@ public Message invoke(Object proxy, final Method method, Object[] args) throws ServiceException { long startTime = 0; if (LOG.isDebugEnabled()) { - startTime = Time.now(); + startTime = Time.monotonicNow(); } if (args.length != 2) { // RpcController + Message @@ -267,7 +267,7 @@ public Message invoke(Object proxy, final Method method, Object[] args) } if (LOG.isDebugEnabled()) { - long callTime = Time.now() - startTime; + long callTime = Time.monotonicNow() - startTime; LOG.debug("Call: " + method.getName() + " took " + callTime + "ms"); } 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 f60590ec38ea8..bdb552bb17964 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 @@ -223,7 +223,7 @@ public Message invoke(Object proxy, final Method method, Object[] args) throws ServiceException { long startTime = 0; if (LOG.isDebugEnabled()) { - startTime = Time.now(); + startTime = Time.monotonicNow(); } if (args.length != 2) { // RpcController + Message @@ -278,7 +278,7 @@ public Message invoke(Object proxy, final Method method, Object[] args) } if (LOG.isDebugEnabled()) { - long callTime = Time.now() - startTime; + long callTime = Time.monotonicNow() - startTime; LOG.debug("Call: " + method.getName() + " took " + callTime + "ms"); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java index 64c486c4b14f8..196874be2ec1d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAsyncIPC.java @@ -28,6 +28,7 @@ import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.Time; import org.apache.hadoop.util.concurrent.AsyncGetFuture; import org.junit.Assert; import org.junit.Before; @@ -38,6 +39,7 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.util.*; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -45,6 +47,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class TestAsyncIPC { @@ -137,6 +141,60 @@ void assertReturnValues(long timeout, TimeUnit unit) } } + /** + * For testing the asynchronous calls of the RPC client + * implemented with CompletableFuture. + */ + static class AsyncCompletableFutureCaller extends Thread { + private final Client client; + private final InetSocketAddress server; + private final int count; + private final List> completableFutures; + private final List expectedValues; + + AsyncCompletableFutureCaller(Client client, InetSocketAddress server, int count) { + this.client = client; + this.server = server; + this.count = count; + this.completableFutures = new ArrayList<>(count); + this.expectedValues = new ArrayList<>(count); + setName("Async CompletableFuture Caller"); + } + + @Override + public void run() { + // Set the RPC client to use asynchronous mode. + Client.setAsynchronousMode(true); + long startTime = Time.monotonicNow(); + try { + for (int i = 0; i < count; i++) { + final long param = TestIPC.RANDOM.nextLong(); + TestIPC.call(client, param, server, conf); + expectedValues.add(param); + completableFutures.add(Client.getResponseFuture()); + } + // Since the run method is asynchronous, + // it does not need to wait for a response after sending a request, + // so the time taken by the run method is less than count * 100 + // (where 100 is the time taken by the server to process a request). + long cost = Time.monotonicNow() - startTime; + assertTrue(cost < count * 100L); + LOG.info("[{}] run cost {}ms", Thread.currentThread().getName(), cost); + } catch (Exception e) { + fail(); + } + } + + public void assertReturnValues() + throws InterruptedException, ExecutionException { + for (int i = 0; i < count; i++) { + LongWritable value = (LongWritable) completableFutures.get(i).get(); + Assert.assertEquals("call" + i + " failed.", + expectedValues.get(i).longValue(), value.get()); + } + } + } + static class AsyncLimitlCaller extends Thread { private Client client; private InetSocketAddress server; @@ -538,4 +596,37 @@ public void run() { assertEquals(startID + i, callIds.get(i).intValue()); } } + + @Test(timeout = 60000) + public void testAsyncCallWithCompletableFuture() throws IOException, + InterruptedException, ExecutionException { + // Override client to store the call id + final Client client = new Client(LongWritable.class, conf); + + // Construct an RPC server, which includes a handler thread. + final TestServer server = new TestIPC.TestServer(1, false, conf); + server.callListener = () -> { + try { + // The server requires at least 100 milliseconds to process a request. + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }; + + try { + InetSocketAddress addr = NetUtils.getConnectAddress(server); + server.start(); + // Send 10 asynchronous requests. + final AsyncCompletableFutureCaller caller = + new AsyncCompletableFutureCaller(client, addr, 10); + caller.start(); + caller.join(); + // Check if the values returned by the asynchronous call meet the expected values. + caller.assertReturnValues(); + } finally { + client.stop(); + server.stop(); + } + } } \ No newline at end of file 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..78d2b312b4f56 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 @@ -274,47 +274,47 @@ public class ClientNamenodeProtocolTranslatorPB implements ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator { final private 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(); @@ -1137,7 +1137,7 @@ public void removeCacheDirective(long id) setId(id).build())); } - private static class BatchedCacheEntries + protected static class BatchedCacheEntries implements BatchedEntries { private final ListCacheDirectivesResponseProto response; @@ -1200,7 +1200,7 @@ public void removeCachePool(String cachePoolName) throws IOException { setPoolName(cachePoolName).build())); } - private static class BatchedCachePoolEntries + protected static class BatchedCachePoolEntries implements BatchedEntries { private final ListCachePoolsResponseProto proto; 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..43bbe0373906c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java @@ -0,0 +1,162 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.hdfs.server.federation.router.ThreadLocalContext; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.ipc.CallerContext; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.ProtobufRpcEngine2; +import org.apache.hadoop.ipc.ProtobufRpcEngineCallback2; +import org.apache.hadoop.ipc.internal.ShadedProtobufHelper; +import org.apache.hadoop.thirdparty.protobuf.Message; +import org.apache.hadoop.util.concurrent.AsyncGet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCompleteWith; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; +import static org.apache.hadoop.ipc.internal.ShadedProtobufHelper.ipc; + +/** + *

This utility class encapsulates the logic required to initiate asynchronous RPCs, + * handle responses, and propagate exceptions. It works in conjunction with + * {@link ProtobufRpcEngine2} and {@link Client} to facilitate the asynchronous + * nature of the operations. + * + * @see ProtobufRpcEngine2 + * @see Client + * @see CompletableFuture + */ +public final class AsyncRpcProtocolPBUtil { + public static final Logger LOG = LoggerFactory.getLogger(AsyncRpcProtocolPBUtil.class); + /** The executor used for handling responses asynchronously. */ + private static Executor asyncResponderExecutor; + + private AsyncRpcProtocolPBUtil() {} + + /** + * Asynchronously invokes an RPC call and applies a response transformation function + * to the result. This method is generic and can be used to handle any type of + * RPC call. + * + *

The method uses the {@link ShadedProtobufHelper.IpcCall} to prepare the RPC call + * and the {@link ApplyFunction} to process the response. It also handles exceptions + * that may occur during the RPC call and wraps them in a user-friendly manner. + * + * @param call The IPC call encapsulating the RPC request. + * @param response The function to apply to the response of the RPC call. + * @param clazz The class object representing the type {@code R} of the response. + * @param Type of the call's result. + * @param Type of method return. + * @return An object of type {@code R} that is the result of applying the response + * function to the RPC call result. + * @throws IOException If an I/O error occurs during the asynchronous RPC call. + */ + public static R asyncIpcClient( + ShadedProtobufHelper.IpcCall call, ApplyFunction response, + Class clazz) throws IOException { + ipc(call); + AsyncGet asyncReqMessage = + (AsyncGet) ProtobufRpcEngine2.getAsyncReturnMessage(); + CompletableFuture responseFuture = Client.getResponseFuture(); + // transfer thread local context to worker threads of executor. + ThreadLocalContext threadLocalContext = new ThreadLocalContext(); + asyncCompleteWith(responseFuture.handleAsync((result, e) -> { + threadLocalContext.transfer(); + if (e != null) { + throw warpCompletionException(e); + } + try { + T res = asyncReqMessage.get(-1, null); + return response.apply(res); + } catch (Exception ex) { + throw warpCompletionException(ex); + } + }, asyncResponderExecutor)); + return asyncReturn(clazz); + } + + /** + * Asynchronously invokes an RPC call and applies a response transformation function + * to the result on server-side. + * @param req The IPC call encapsulating the RPC request on server-side. + * @param res The function to apply to the response of the RPC call on server-side. + * @param Type of the call's result. + */ + public static void asyncRouterServer(ServerReq req, ServerRes res) { + final ProtobufRpcEngineCallback2 callback = + ProtobufRpcEngine2.Server.registerForDeferredResponse2(); + + CompletableFuture completableFuture = + CompletableFuture.completedFuture(null); + completableFuture.thenCompose(o -> { + try { + req.req(); + return (CompletableFuture) AsyncUtil.getAsyncUtilCompletableFuture(); + } catch (Exception e) { + throw new CompletionException(e); + } + }).handle((result, e) -> { + LOG.debug("Async response, callback: {}, CallerContext: {}, result: [{}], exception: [{}]", + callback, CallerContext.getCurrent(), result, e); + if (e == null) { + Message value = null; + try { + value = res.res(result); + } catch (Exception re) { + callback.error(re); + return null; + } + callback.setResponse(value); + } else { + callback.error(e.getCause()); + } + return null; + }); + } + + /** + * Sets the executor used for handling responses asynchronously within + * the utility class. + * + * @param asyncResponderExecutor The executor to be used for handling responses asynchronously. + */ + public static void setAsyncResponderExecutor(Executor asyncResponderExecutor) { + AsyncRpcProtocolPBUtil.asyncResponderExecutor = asyncResponderExecutor; + } + + @FunctionalInterface + interface ServerReq { + T req() throws Exception; + } + + @FunctionalInterface + interface ServerRes { + Message res(T result) throws Exception; + } +} 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..e70240681d27e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientNamenodeProtocolServerSideTranslatorPB.java @@ -0,0 +1,1769 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.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.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.ModifyAclEntriesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos; +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.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.FinalizeUpgradeRequestProto; +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.GetDataEncryptionKeyRequestProto; +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.GetFsECBlockGroupStatsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatusRequestProto; +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.GetSnapshotDiffReportListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto; +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.GetSnapshotListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingResponseProto; +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.GetStoragePoliciesRequestProto; +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.RecoverLeaseRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto; +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.RollEditsRequestProto; +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.SatisfyStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyResponseProto; +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.UpgradeStatusRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto; +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.ReencryptEncryptionZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptEncryptionZoneResponseProto; +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.DisableErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.DisableErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.EnableErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.EnableErasureCodingPolicyResponseProto; +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.ErasureCodingProtos.GetErasureCodingCodecsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingCodecsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto; +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.RemoveErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrResponseProto; +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.CancelDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto; +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.proto.SecurityProtos.RenewDelegationTokenResponseProto; +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 GetBlockLocationsResponseProto getBlockLocations( + RpcController controller, GetBlockLocationsRequestProto req) { + asyncRouterServer(() -> server.getBlockLocations(req.getSrc(), req.getOffset(), + req.getLength()), + b -> { + GetBlockLocationsResponseProto.Builder builder + = GetBlockLocationsResponseProto + .newBuilder(); + if (b != null) { + builder.setLocations(PBHelperClient.convert(b)).build(); + } + return builder.build(); + }); + return null; + } + + @Override + public GetServerDefaultsResponseProto getServerDefaults( + RpcController controller, GetServerDefaultsRequestProto req) { + asyncRouterServer(server::getServerDefaults, + result -> GetServerDefaultsResponseProto.newBuilder() + .setServerDefaults(PBHelperClient.convert(result)) + .build()); + return null; + } + + + @Override + public CreateResponseProto create(RpcController controller, 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 CreateResponseProto.newBuilder().setFs(PBHelperClient.convert(result)) + .build(); + } + return VOID_CREATE_RESPONSE; + }); + return null; + } + + @Override + public AppendResponseProto append(RpcController controller, + 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 -> { + AppendResponseProto.Builder builder = + 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 SetReplicationResponseProto setReplication( + RpcController controller, + SetReplicationRequestProto req) { + asyncRouterServer(() -> + server.setReplication(req.getSrc(), (short) req.getReplication()), + result -> SetReplicationResponseProto.newBuilder().setResult(result).build()); + return null; + } + + + @Override + public SetPermissionResponseProto setPermission( + RpcController controller, + SetPermissionRequestProto req) { + asyncRouterServer(() -> { + server.setPermission(req.getSrc(), PBHelperClient.convert(req.getPermission())); + return null; + }, result -> VOID_SET_PERM_RESPONSE); + return null; + } + + @Override + public SetOwnerResponseProto setOwner( + RpcController controller, + 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 AbandonBlockResponseProto abandonBlock( + RpcController controller, + 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 AddBlockResponseProto addBlock( + RpcController controller, + 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 -> AddBlockResponseProto.newBuilder() + .setBlock(PBHelperClient.convertLocatedBlock(result)).build()); + return null; + } + + @Override + public GetAdditionalDatanodeResponseProto getAdditionalDatanode( + RpcController controller, 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 -> GetAdditionalDatanodeResponseProto.newBuilder() + .setBlock( + PBHelperClient.convertLocatedBlock(result)) + .build()); + return null; + } + + @Override + public CompleteResponseProto complete( + RpcController controller, + 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 -> CompleteResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public ReportBadBlocksResponseProto reportBadBlocks( + RpcController controller, + 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 ConcatResponseProto concat( + RpcController controller, + 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 RenameResponseProto rename( + RpcController controller, + RenameRequestProto req) { + asyncRouterServer(() -> { + return server.rename(req.getSrc(), req.getDst()); + }, result -> RenameResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public Rename2ResponseProto rename2( + RpcController controller, + 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 TruncateResponseProto truncate( + RpcController controller, + TruncateRequestProto req) { + asyncRouterServer(() -> server.truncate(req.getSrc(), req.getNewLength(), + req.getClientName()), + result -> TruncateResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public DeleteResponseProto delete( + RpcController controller, + DeleteRequestProto req) { + asyncRouterServer(() -> server.delete(req.getSrc(), req.getRecursive()), + result -> DeleteResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public MkdirsResponseProto mkdirs( + RpcController controller, + 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 -> MkdirsResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public GetListingResponseProto getListing( + RpcController controller, + GetListingRequestProto req) { + asyncRouterServer(() -> { + DirectoryListing result = server.getListing( + req.getSrc(), req.getStartAfter().toByteArray(), + req.getNeedLocation()); + return result; + }, result -> { + if (result != null) { + return GetListingResponseProto.newBuilder().setDirList( + PBHelperClient.convert(result)).build(); + } else { + return VOID_GETLISTING_RESPONSE; + } + }); + return null; + } + + @Override + public GetBatchedListingResponseProto getBatchedListing( + RpcController controller, + GetBatchedListingRequestProto request) { + asyncRouterServer(() -> { + BatchedDirectoryListing result = server.getBatchedListing( + request.getPathsList().toArray(new String[]{}), + request.getStartAfter().toByteArray(), + request.getNeedLocation()); + return result; + }, result -> { + if (result != null) { + GetBatchedListingResponseProto.Builder builder = + 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 RenewLeaseResponseProto renewLease( + RpcController controller, + RenewLeaseRequestProto req) { + asyncRouterServer(() -> { + server.renewLease(req.getClientName(), req.getNamespacesList()); + return null; + }, result -> VOID_RENEWLEASE_RESPONSE); + return null; + } + + @Override + public RecoverLeaseResponseProto recoverLease( + RpcController controller, + RecoverLeaseRequestProto req) { + asyncRouterServer(() -> server.recoverLease(req.getSrc(), req.getClientName()), + result -> RecoverLeaseResponseProto.newBuilder() + .setResult(result).build()); + return null; + } + + @Override + public RestoreFailedStorageResponseProto restoreFailedStorage( + RpcController controller, RestoreFailedStorageRequestProto req) { + asyncRouterServer(() -> server.restoreFailedStorage(req.getArg()), + result -> RestoreFailedStorageResponseProto.newBuilder().setResult(result) + .build()); + return null; + } + + @Override + public GetFsStatsResponseProto getFsStats( + RpcController controller, + GetFsStatusRequestProto req) { + asyncRouterServer(server::getStats, PBHelperClient::convert); + return null; + } + + @Override + public GetFsReplicatedBlockStatsResponseProto getFsReplicatedBlockStats( + RpcController controller, GetFsReplicatedBlockStatsRequestProto request) { + asyncRouterServer(server::getReplicatedBlockStats, PBHelperClient::convert); + return null; + } + + @Override + public GetFsECBlockGroupStatsResponseProto getFsECBlockGroupStats( + RpcController controller, GetFsECBlockGroupStatsRequestProto request) { + asyncRouterServer(server::getECBlockGroupStats, PBHelperClient::convert); + return null; + } + + @Override + public GetDatanodeReportResponseProto getDatanodeReport( + RpcController controller, GetDatanodeReportRequestProto req) { + asyncRouterServer(() -> server.getDatanodeReport(PBHelperClient.convert(req.getType())), + result -> { + List re = PBHelperClient.convert(result); + return GetDatanodeReportResponseProto.newBuilder() + .addAllDi(re).build(); + }); + return null; + } + + @Override + public GetDatanodeStorageReportResponseProto getDatanodeStorageReport( + RpcController controller, GetDatanodeStorageReportRequestProto req) { + asyncRouterServer(() -> server.getDatanodeStorageReport(PBHelperClient.convert(req.getType())), + result -> { + List reports = + PBHelperClient.convertDatanodeStorageReports(result); + return GetDatanodeStorageReportResponseProto.newBuilder() + .addAllDatanodeStorageReports(reports) + .build(); + }); + return null; + } + + @Override + public GetPreferredBlockSizeResponseProto getPreferredBlockSize( + RpcController controller, GetPreferredBlockSizeRequestProto req) { + asyncRouterServer(() -> server.getPreferredBlockSize(req.getFilename()), + result -> GetPreferredBlockSizeResponseProto.newBuilder().setBsize(result) + .build()); + return null; + } + + @Override + public SetSafeModeResponseProto setSafeMode( + RpcController controller, + SetSafeModeRequestProto req) { + asyncRouterServer(() -> server.setSafeMode(PBHelperClient.convert(req.getAction()), + req.getChecked()), + result -> SetSafeModeResponseProto.newBuilder().setResult(result).build()); + return null; + } + + @Override + public SaveNamespaceResponseProto saveNamespace( + RpcController controller, + 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 -> SaveNamespaceResponseProto.newBuilder().setSaved(result).build()); + return null; + } + + @Override + public RollEditsResponseProto rollEdits( + RpcController controller, + RollEditsRequestProto request) { + asyncRouterServer(server::rollEdits, + txid -> RollEditsResponseProto.newBuilder() + .setNewSegmentTxId(txid) + .build()); + return null; + } + + + @Override + public RefreshNodesResponseProto refreshNodes( + RpcController controller, + RefreshNodesRequestProto req) { + asyncRouterServer(() -> { + server.refreshNodes(); + return null; + }, result -> VOID_REFRESHNODES_RESPONSE); + return null; + } + + @Override + public FinalizeUpgradeResponseProto finalizeUpgrade( + RpcController controller, + FinalizeUpgradeRequestProto req) { + asyncRouterServer(() -> { + server.finalizeUpgrade(); + return null; + }, result -> VOID_REFRESHNODES_RESPONSE); + return null; + } + + @Override + public UpgradeStatusResponseProto upgradeStatus( + RpcController controller, UpgradeStatusRequestProto req) { + asyncRouterServer(server::upgradeStatus, + result -> { + UpgradeStatusResponseProto.Builder b = + UpgradeStatusResponseProto.newBuilder(); + b.setUpgradeFinalized(result); + return b.build(); + }); + return null; + } + + @Override + public RollingUpgradeResponseProto rollingUpgrade( + RpcController controller, + RollingUpgradeRequestProto req) { + asyncRouterServer(() -> + server.rollingUpgrade(PBHelperClient.convert(req.getAction())), + info -> { + final RollingUpgradeResponseProto.Builder b = + RollingUpgradeResponseProto.newBuilder(); + if (info != null) { + b.setRollingUpgradeInfo(PBHelperClient.convert(info)); + } + return b.build(); + }); + return null; + } + + @Override + public ListCorruptFileBlocksResponseProto listCorruptFileBlocks( + RpcController controller, ListCorruptFileBlocksRequestProto req) { + asyncRouterServer(() -> server.listCorruptFileBlocks( + req.getPath(), req.hasCookie() ? req.getCookie(): null), + result -> ListCorruptFileBlocksResponseProto.newBuilder() + .setCorrupt(PBHelperClient.convert(result)) + .build()); + return null; + } + + @Override + public MetaSaveResponseProto metaSave( + RpcController controller, + MetaSaveRequestProto req) { + asyncRouterServer(() -> { + server.metaSave(req.getFilename()); + return null; + }, result -> VOID_METASAVE_RESPONSE); + return null; + } + + @Override + public GetFileInfoResponseProto getFileInfo( + RpcController controller, + GetFileInfoRequestProto req) { + asyncRouterServer(() -> server.getFileInfo(req.getSrc()), + result -> { + if (result != null) { + return GetFileInfoResponseProto.newBuilder().setFs( + PBHelperClient.convert(result)).build(); + } + return VOID_GETFILEINFO_RESPONSE; + }); + return null; + } + + @Override + public GetLocatedFileInfoResponseProto getLocatedFileInfo( + RpcController controller, GetLocatedFileInfoRequestProto req) { + asyncRouterServer(() -> server.getLocatedFileInfo(req.getSrc(), + req.getNeedBlockToken()), + result -> { + if (result != null) { + return GetLocatedFileInfoResponseProto.newBuilder().setFs( + PBHelperClient.convert(result)).build(); + } + return VOID_GETLOCATEDFILEINFO_RESPONSE; + }); + return null; + } + + @Override + public GetFileLinkInfoResponseProto getFileLinkInfo( + RpcController controller, + GetFileLinkInfoRequestProto req) { + asyncRouterServer(() -> server.getFileLinkInfo(req.getSrc()), + result -> { + if (result != null) { + return GetFileLinkInfoResponseProto.newBuilder().setFs( + PBHelperClient.convert(result)).build(); + } else { + return VOID_GETFILELINKINFO_RESPONSE; + } + }); + return null; + } + + @Override + public GetContentSummaryResponseProto getContentSummary( + RpcController controller, GetContentSummaryRequestProto req) { + asyncRouterServer(() -> server.getContentSummary(req.getPath()), + result -> GetContentSummaryResponseProto.newBuilder() + .setSummary(PBHelperClient.convert(result)).build()); + return null; + } + + @Override + public SetQuotaResponseProto setQuota( + RpcController controller, + SetQuotaRequestProto req) { + 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 FsyncResponseProto fsync( + RpcController controller, + FsyncRequestProto req) { + asyncRouterServer(() -> { + server.fsync(req.getSrc(), req.getFileId(), + req.getClient(), req.getLastBlockLength()); + return null; + }, result -> VOID_FSYNC_RESPONSE); + return null; + } + + @Override + public SetTimesResponseProto setTimes( + RpcController controller, + SetTimesRequestProto req) { + asyncRouterServer(() -> { + server.setTimes(req.getSrc(), req.getMtime(), req.getAtime()); + return null; + }, result -> VOID_SETTIMES_RESPONSE); + return null; + } + + @Override + public CreateSymlinkResponseProto createSymlink( + RpcController controller, + 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 GetLinkTargetResponseProto getLinkTarget( + RpcController controller, + GetLinkTargetRequestProto req) { + asyncRouterServer(() -> server.getLinkTarget(req.getPath()), + result -> { + GetLinkTargetResponseProto.Builder builder = + GetLinkTargetResponseProto + .newBuilder(); + if (result != null) { + builder.setTargetPath(result); + } + return builder.build(); + }); + return null; + } + + @Override + public UpdateBlockForPipelineResponseProto updateBlockForPipeline( + RpcController controller, UpdateBlockForPipelineRequestProto req) { + asyncRouterServer(() -> server.updateBlockForPipeline(PBHelperClient.convert(req.getBlock()), + req.getClientName()), + result -> { + HdfsProtos.LocatedBlockProto res = PBHelperClient.convertLocatedBlock(result); + return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(res) + .build(); + }); + return null; + } + + @Override + public UpdatePipelineResponseProto updatePipeline( + RpcController controller, + 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 GetDelegationTokenResponseProto getDelegationToken( + RpcController controller, GetDelegationTokenRequestProto req) { + asyncRouterServer(() -> server + .getDelegationToken(new Text(req.getRenewer())), + token -> { + GetDelegationTokenResponseProto.Builder rspBuilder = + GetDelegationTokenResponseProto.newBuilder(); + if (token != null) { + rspBuilder.setToken(PBHelperClient.convert(token)); + } + return rspBuilder.build(); + }); + return null; + } + + @Override + public RenewDelegationTokenResponseProto renewDelegationToken( + RpcController controller, RenewDelegationTokenRequestProto req) { + asyncRouterServer(() -> server.renewDelegationToken(PBHelperClient + .convertDelegationToken(req.getToken())), + result -> RenewDelegationTokenResponseProto.newBuilder() + .setNewExpiryTime(result).build()); + return null; + } + + @Override + public CancelDelegationTokenResponseProto cancelDelegationToken( + RpcController controller, CancelDelegationTokenRequestProto req) { + asyncRouterServer(() -> { + server.cancelDelegationToken(PBHelperClient.convertDelegationToken(req + .getToken())); + return null; + }, result -> VOID_CANCELDELEGATIONTOKEN_RESPONSE); + return null; + } + + @Override + public SetBalancerBandwidthResponseProto setBalancerBandwidth( + RpcController controller, SetBalancerBandwidthRequestProto req) { + asyncRouterServer(() -> { + server.setBalancerBandwidth(req.getBandwidth()); + return null; + }, result -> VOID_SETBALANCERBANDWIDTH_RESPONSE); + return null; + } + + @Override + public GetDataEncryptionKeyResponseProto getDataEncryptionKey( + RpcController controller, GetDataEncryptionKeyRequestProto request) { + asyncRouterServer(server::getDataEncryptionKey, encryptionKey -> { + GetDataEncryptionKeyResponseProto.Builder builder = + GetDataEncryptionKeyResponseProto.newBuilder(); + if (encryptionKey != null) { + builder.setDataEncryptionKey(PBHelperClient.convert(encryptionKey)); + } + return builder.build(); + }); + return null; + } + + @Override + public CreateSnapshotResponseProto createSnapshot( + RpcController controller, + CreateSnapshotRequestProto req) throws ServiceException { + asyncRouterServer(() -> server.createSnapshot(req.getSnapshotRoot(), + req.hasSnapshotName()? req.getSnapshotName(): null), + snapshotPath -> { + final CreateSnapshotResponseProto.Builder builder + = CreateSnapshotResponseProto.newBuilder(); + if (snapshotPath != null) { + builder.setSnapshotPath(snapshotPath); + } + return builder.build(); + }); + return null; + } + + @Override + public DeleteSnapshotResponseProto deleteSnapshot( + RpcController controller, + DeleteSnapshotRequestProto req) { + asyncRouterServer(() -> { + server.deleteSnapshot(req.getSnapshotRoot(), req.getSnapshotName()); + return null; + }, result -> VOID_DELETE_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public AllowSnapshotResponseProto allowSnapshot( + RpcController controller, + AllowSnapshotRequestProto req) { + asyncRouterServer(() -> { + server.allowSnapshot(req.getSnapshotRoot()); + return null; + }, result -> VOID_ALLOW_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public DisallowSnapshotResponseProto disallowSnapshot( + RpcController controller, + DisallowSnapshotRequestProto req) { + asyncRouterServer(() -> { + server.disallowSnapshot(req.getSnapshotRoot()); + return null; + }, result -> VOID_DISALLOW_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public RenameSnapshotResponseProto renameSnapshot( + RpcController controller, + RenameSnapshotRequestProto request) { + asyncRouterServer(() -> { + server.renameSnapshot(request.getSnapshotRoot(), + request.getSnapshotOldName(), request.getSnapshotNewName()); + return null; + }, result -> VOID_RENAME_SNAPSHOT_RESPONSE); + return null; + } + + @Override + public GetSnapshottableDirListingResponseProto getSnapshottableDirListing( + RpcController controller, GetSnapshottableDirListingRequestProto request) { + asyncRouterServer(server::getSnapshottableDirListing, + result -> { + if (result != null) { + return GetSnapshottableDirListingResponseProto.newBuilder(). + setSnapshottableDirList(PBHelperClient.convert(result)).build(); + } else { + return NULL_GET_SNAPSHOTTABLE_DIR_LISTING_RESPONSE; + } + }); + return null; + } + + @Override + public GetSnapshotListingResponseProto getSnapshotListing( + RpcController controller, GetSnapshotListingRequestProto request) { + asyncRouterServer(() -> server + .getSnapshotListing(request.getSnapshotRoot()), + result -> { + if (result != null) { + return GetSnapshotListingResponseProto.newBuilder(). + setSnapshotList(PBHelperClient.convert(result)).build(); + } else { + return NULL_GET_SNAPSHOT_LISTING_RESPONSE; + } + }); + return null; + } + + @Override + public GetSnapshotDiffReportResponseProto getSnapshotDiffReport( + RpcController controller, GetSnapshotDiffReportRequestProto request) { + asyncRouterServer(() -> server.getSnapshotDiffReport( + request.getSnapshotRoot(), request.getFromSnapshot(), + request.getToSnapshot()), + report -> GetSnapshotDiffReportResponseProto.newBuilder() + .setDiffReport(PBHelperClient.convert(report)).build()); + return null; + } + + @Override + public GetSnapshotDiffReportListingResponseProto getSnapshotDiffReportListing( + RpcController controller, + GetSnapshotDiffReportListingRequestProto request) { + asyncRouterServer(() -> server + .getSnapshotDiffReportListing(request.getSnapshotRoot(), + request.getFromSnapshot(), request.getToSnapshot(), + request.getCursor().getStartPath().toByteArray(), + request.getCursor().getIndex()), + report -> GetSnapshotDiffReportListingResponseProto.newBuilder() + .setDiffReport(PBHelperClient.convert(report)).build()); + return null; + } + + @Override + public IsFileClosedResponseProto isFileClosed( + RpcController controller, IsFileClosedRequestProto request) { + asyncRouterServer(() -> server.isFileClosed(request.getSrc()), + result -> IsFileClosedResponseProto.newBuilder() + .setResult(result).build()); + return null; + } + + @Override + public AddCacheDirectiveResponseProto addCacheDirective( + RpcController controller, AddCacheDirectiveRequestProto request) { + asyncRouterServer(() -> server.addCacheDirective( + PBHelperClient.convert(request.getInfo()), + PBHelperClient.convertCacheFlags(request.getCacheFlags())), + id -> AddCacheDirectiveResponseProto.newBuilder(). + setId(id).build()); + return null; + } + + @Override + public ModifyCacheDirectiveResponseProto modifyCacheDirective( + RpcController controller, ModifyCacheDirectiveRequestProto request) { + asyncRouterServer(() -> { + server.modifyCacheDirective( + PBHelperClient.convert(request.getInfo()), + PBHelperClient.convertCacheFlags(request.getCacheFlags())); + return null; + }, result -> ModifyCacheDirectiveResponseProto.newBuilder().build()); + return null; + } + + @Override + public RemoveCacheDirectiveResponseProto removeCacheDirective( + RpcController controller, + RemoveCacheDirectiveRequestProto request) { + asyncRouterServer(() -> { + server.removeCacheDirective(request.getId()); + return null; + }, result -> RemoveCacheDirectiveResponseProto. + newBuilder().build()); + return null; + } + + @Override + public ListCacheDirectivesResponseProto listCacheDirectives( + RpcController controller, ListCacheDirectivesRequestProto request) { + asyncRouterServer(() -> { + CacheDirectiveInfo filter = + PBHelperClient.convert(request.getFilter()); + return server.listCacheDirectives(request.getPrevId(), filter); + }, entries -> { + ListCacheDirectivesResponseProto.Builder builder = + ListCacheDirectivesResponseProto.newBuilder(); + builder.setHasMore(entries.hasMore()); + for (int i=0, n=entries.size(); i { + server.addCachePool(PBHelperClient.convert(request.getInfo())); + return null; + }, result -> AddCachePoolResponseProto.newBuilder().build()); + return null; + } + + @Override + public ModifyCachePoolResponseProto modifyCachePool( + RpcController controller, + ModifyCachePoolRequestProto request) { + asyncRouterServer(() -> { + server.modifyCachePool(PBHelperClient.convert(request.getInfo())); + return null; + }, result -> ModifyCachePoolResponseProto.newBuilder().build()); + return null; + } + + @Override + public RemoveCachePoolResponseProto removeCachePool( + RpcController controller, + RemoveCachePoolRequestProto request) { + asyncRouterServer(() -> { + server.removeCachePool(request.getPoolName()); + return null; + }, result -> RemoveCachePoolResponseProto.newBuilder().build()); + return null; + } + + @Override + public ListCachePoolsResponseProto listCachePools( + RpcController controller, + ListCachePoolsRequestProto request) { + asyncRouterServer(() -> server.listCachePools(request.getPrevPoolName()), + entries -> { + ListCachePoolsResponseProto.Builder responseBuilder = + 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 RemoveAclEntriesResponseProto removeAclEntries( + RpcController controller, RemoveAclEntriesRequestProto req) { + asyncRouterServer(() -> { + server.removeAclEntries(req.getSrc(), + PBHelperClient.convertAclEntry(req.getAclSpecList())); + return null; + }, vo -> VOID_REMOVEACLENTRIES_RESPONSE); + return null; + } + + @Override + public RemoveDefaultAclResponseProto removeDefaultAcl( + RpcController controller, RemoveDefaultAclRequestProto req) { + asyncRouterServer(() -> { + server.removeDefaultAcl(req.getSrc()); + return null; + }, vo -> VOID_REMOVEDEFAULTACL_RESPONSE); + return null; + } + + @Override + public RemoveAclResponseProto removeAcl( + RpcController controller, + RemoveAclRequestProto req) { + asyncRouterServer(() -> { + server.removeAcl(req.getSrc()); + return null; + }, vo -> VOID_REMOVEACL_RESPONSE); + return null; + } + + @Override + public SetAclResponseProto setAcl( + RpcController controller, + SetAclRequestProto req) { + asyncRouterServer(() -> { + server.setAcl(req.getSrc(), PBHelperClient.convertAclEntry(req.getAclSpecList())); + return null; + }, vo -> VOID_SETACL_RESPONSE); + return null; + } + + @Override + public GetAclStatusResponseProto getAclStatus( + RpcController controller, + GetAclStatusRequestProto req) { + asyncRouterServer(() -> server.getAclStatus(req.getSrc()), + PBHelperClient::convert); + return null; + } + + @Override + public CreateEncryptionZoneResponseProto createEncryptionZone( + RpcController controller, CreateEncryptionZoneRequestProto req) { + asyncRouterServer(() -> { + server.createEncryptionZone(req.getSrc(), req.getKeyName()); + return null; + }, vo -> CreateEncryptionZoneResponseProto.newBuilder().build()); + return null; + } + + @Override + public GetEZForPathResponseProto getEZForPath( + RpcController controller, GetEZForPathRequestProto req) { + asyncRouterServer(() -> server.getEZForPath(req.getSrc()), + ret -> { + GetEZForPathResponseProto.Builder builder = + GetEZForPathResponseProto.newBuilder(); + if (ret != null) { + builder.setZone(PBHelperClient.convert(ret)); + } + return builder.build(); + }); + return null; + } + + @Override + public ListEncryptionZonesResponseProto listEncryptionZones( + RpcController controller, ListEncryptionZonesRequestProto req) { + asyncRouterServer(() -> server.listEncryptionZones(req.getId()), + entries -> { + ListEncryptionZonesResponseProto.Builder builder = + ListEncryptionZonesResponseProto.newBuilder(); + builder.setHasMore(entries.hasMore()); + for (int i=0; i { + server.reencryptEncryptionZone(req.getZone(), + PBHelperClient.convert(req.getAction())); + return null; + }, vo -> ReencryptEncryptionZoneResponseProto.newBuilder().build()); + return null; + } + + public ListReencryptionStatusResponseProto listReencryptionStatus( + RpcController controller, ListReencryptionStatusRequestProto req) { + asyncRouterServer(() -> server.listReencryptionStatus(req.getId()), + entries -> { + ListReencryptionStatusResponseProto.Builder builder = + 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 -> SetErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public UnsetErasureCodingPolicyResponseProto unsetErasureCodingPolicy( + RpcController controller, UnsetErasureCodingPolicyRequestProto req) { + asyncRouterServer(() -> { + server.unsetErasureCodingPolicy(req.getSrc()); + return null; + }, vo -> UnsetErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public GetECTopologyResultForPoliciesResponseProto getECTopologyResultForPolicies( + RpcController controller, GetECTopologyResultForPoliciesRequestProto req) { + asyncRouterServer(() -> { + ProtocolStringList policies = req.getPoliciesList(); + return server.getECTopologyResultForPolicies( + policies.toArray(policies.toArray(new String[policies.size()]))); + }, result -> { + GetECTopologyResultForPoliciesResponseProto.Builder builder = + GetECTopologyResultForPoliciesResponseProto.newBuilder(); + builder + .setResponse(PBHelperClient.convertECTopologyVerifierResult(result)); + return builder.build(); + }); + return null; + } + + @Override + public SetXAttrResponseProto setXAttr( + RpcController controller, + 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 GetXAttrsResponseProto getXAttrs( + RpcController controller, + GetXAttrsRequestProto req) { + asyncRouterServer(() -> server.getXAttrs(req.getSrc(), + PBHelperClient.convertXAttrs(req.getXAttrsList())), + PBHelperClient::convertXAttrsResponse); + return null; + } + + @Override + public ListXAttrsResponseProto listXAttrs( + RpcController controller, + ListXAttrsRequestProto req) { + asyncRouterServer(() -> server.listXAttrs(req.getSrc()), + PBHelperClient::convertListXAttrsResponse); + return null; + } + + @Override + public RemoveXAttrResponseProto removeXAttr( + RpcController controller, + RemoveXAttrRequestProto req) { + asyncRouterServer(() -> { + server.removeXAttr(req.getSrc(), PBHelperClient.convertXAttr(req.getXAttr())); + return null; + }, vo -> VOID_REMOVEXATTR_RESPONSE); + return null; + } + + @Override + public CheckAccessResponseProto checkAccess( + RpcController controller, + CheckAccessRequestProto req) { + asyncRouterServer(() -> { + server.checkAccess(req.getPath(), PBHelperClient.convert(req.getMode())); + return null; + }, vo -> VOID_CHECKACCESS_RESPONSE); + return null; + } + + @Override + public SetStoragePolicyResponseProto setStoragePolicy( + RpcController controller, SetStoragePolicyRequestProto request) { + asyncRouterServer(() -> { + server.setStoragePolicy(request.getSrc(), request.getPolicyName()); + return null; + }, vo -> VOID_SET_STORAGE_POLICY_RESPONSE); + return null; + } + + @Override + public UnsetStoragePolicyResponseProto unsetStoragePolicy( + RpcController controller, UnsetStoragePolicyRequestProto request) { + asyncRouterServer(() -> { + server.unsetStoragePolicy(request.getSrc()); + return null; + }, vo -> VOID_UNSET_STORAGE_POLICY_RESPONSE); + return null; + } + + @Override + public GetStoragePolicyResponseProto getStoragePolicy( + RpcController controller, GetStoragePolicyRequestProto request) { + asyncRouterServer(() -> server.getStoragePolicy(request.getPath()), + result -> { + HdfsProtos.BlockStoragePolicyProto policy = PBHelperClient.convert(result); + return GetStoragePolicyResponseProto.newBuilder() + .setStoragePolicy(policy).build(); + }); + return null; + } + + @Override + public GetStoragePoliciesResponseProto getStoragePolicies( + RpcController controller, GetStoragePoliciesRequestProto request) { + asyncRouterServer(server::getStoragePolicies, + policies -> { + GetStoragePoliciesResponseProto.Builder builder = + GetStoragePoliciesResponseProto.newBuilder(); + if (policies == null) { + return builder.build(); + } + for (BlockStoragePolicy policy : policies) { + builder.addPolicies(PBHelperClient.convert(policy)); + } + return builder.build(); + }); + return null; + } + + public GetCurrentEditLogTxidResponseProto getCurrentEditLogTxid( + RpcController controller, + GetCurrentEditLogTxidRequestProto req) { + asyncRouterServer(server::getCurrentEditLogTxid, + result -> GetCurrentEditLogTxidResponseProto.newBuilder() + .setTxid(result).build()); + return null; + } + + @Override + public GetEditsFromTxidResponseProto getEditsFromTxid( + RpcController controller, + GetEditsFromTxidRequestProto req) { + asyncRouterServer(() -> server.getEditsFromTxid(req.getTxid()), + PBHelperClient::convertEditsResponse); + return null; + } + + @Override + public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies( + RpcController controller, + GetErasureCodingPoliciesRequestProto request) { + asyncRouterServer(server::getErasureCodingPolicies, + ecpInfos -> { + GetErasureCodingPoliciesResponseProto.Builder resBuilder = + GetErasureCodingPoliciesResponseProto + .newBuilder(); + for (ErasureCodingPolicyInfo info : ecpInfos) { + resBuilder.addEcPolicies( + PBHelperClient.convertErasureCodingPolicy(info)); + } + return resBuilder.build(); + }); + return null; + } + + @Override + public GetErasureCodingCodecsResponseProto getErasureCodingCodecs( + RpcController controller, GetErasureCodingCodecsRequestProto request) { + asyncRouterServer(server::getErasureCodingCodecs, + codecs -> { + GetErasureCodingCodecsResponseProto.Builder resBuilder = + GetErasureCodingCodecsResponseProto.newBuilder(); + for (Map.Entry codec : codecs.entrySet()) { + resBuilder.addCodec( + PBHelperClient.convertErasureCodingCodec( + codec.getKey(), codec.getValue())); + } + return resBuilder.build(); + }); + return null; + } + + @Override + public AddErasureCodingPoliciesResponseProto addErasureCodingPolicies( + RpcController controller, 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()); + AddErasureCodingPoliciesResponseProto response = + AddErasureCodingPoliciesResponseProto.newBuilder() + .addAllResponses(responseProtos).build(); + return response; + }); + return null; + } + + @Override + public RemoveErasureCodingPolicyResponseProto removeErasureCodingPolicy( + RpcController controller, RemoveErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> { + server.removeErasureCodingPolicy(request.getEcPolicyName()); + return null; + }, vo -> RemoveErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public EnableErasureCodingPolicyResponseProto enableErasureCodingPolicy( + RpcController controller, EnableErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> { + server.enableErasureCodingPolicy(request.getEcPolicyName()); + return null; + }, vo -> EnableErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public DisableErasureCodingPolicyResponseProto disableErasureCodingPolicy( + RpcController controller, DisableErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> { + server.disableErasureCodingPolicy(request.getEcPolicyName()); + return null; + }, vo -> DisableErasureCodingPolicyResponseProto.newBuilder().build()); + return null; + } + + @Override + public GetErasureCodingPolicyResponseProto getErasureCodingPolicy( + RpcController controller, + GetErasureCodingPolicyRequestProto request) { + asyncRouterServer(() -> server.getErasureCodingPolicy(request.getSrc()), + ecPolicy -> { + GetErasureCodingPolicyResponseProto.Builder builder = + GetErasureCodingPolicyResponseProto.newBuilder(); + if (ecPolicy != null) { + builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy)); + } + return builder.build(); + }); + return null; + } + + @Override + public GetQuotaUsageResponseProto getQuotaUsage( + RpcController controller, GetQuotaUsageRequestProto req) { + asyncRouterServer(() -> server.getQuotaUsage(req.getPath()), + result -> GetQuotaUsageResponseProto.newBuilder() + .setUsage(PBHelperClient.convert(result)).build()); + return null; + } + + @Override + public ListOpenFilesResponseProto listOpenFiles( + RpcController controller, + ListOpenFilesRequestProto req) { + asyncRouterServer(() -> { + EnumSet openFilesTypes = + PBHelperClient.convertOpenFileTypes(req.getTypesList()); + return server.listOpenFiles(req.getId(), + openFilesTypes, req.getPath()); + }, entries -> { + ListOpenFilesResponseProto.Builder builder = + 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 MsyncResponseProto msync(RpcController controller, MsyncRequestProto req) { + asyncRouterServer(() -> { + server.msync(); + return null; + }, vo -> MsyncResponseProto.newBuilder().build()); + return null; + } + + @Override + public SatisfyStoragePolicyResponseProto satisfyStoragePolicy( + RpcController controller, SatisfyStoragePolicyRequestProto request) { + asyncRouterServer(() -> { + server.satisfyStoragePolicy(request.getSrc()); + return null; + }, vo -> VOID_SATISFYSTORAGEPOLICY_RESPONSE); + return null; + } + + @Override + public HAServiceStateResponseProto getHAServiceState( + RpcController controller, + 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; + } + HAServiceStateResponseProto.Builder builder = + HAServiceStateResponseProto.newBuilder(); + builder.setState(retState); + return builder.build(); + }); + return null; + } + + @Override + public GetSlowDatanodeReportResponseProto getSlowDatanodeReport( + RpcController controller, + GetSlowDatanodeReportRequestProto request) { + asyncRouterServer(server::getSlowDatanodeReport, + res -> { + List result = + PBHelperClient.convert(res); + return GetSlowDatanodeReportResponseProto.newBuilder() + .addAllDatanodeInfoProto(result) + .build(); + }); + return null; + } + + @Override + public GetEnclosingRootResponseProto getEnclosingRoot( + RpcController controller, 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..93cbae4ec61ee --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientProtocolTranslatorPB.java @@ -0,0 +1,2025 @@ +/** + * 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.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.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.GetAclStatusRequestProto; +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.AbandonBlockRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBatchedListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSlowDatanodeReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListOpenFilesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncRequestProto; +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.RemoveCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyRequestProto; +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.ZoneReencryptionStatusProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptEncryptionZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto; +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.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.HdfsProtos.ErasureCodingPolicyProto; +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.RemoteException; +import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; +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.util.Lists; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpcClient; + +/** + * 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 { + private final ClientNamenodeProtocolPB rpcProxy; + + public RouterClientProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) { + super(proxy); + rpcProxy = 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(); + + return asyncIpcClient(() -> rpcProxy.getBlockLocations(null, req), + res -> res.hasLocations() ? PBHelperClient.convert(res.getLocations()) : null, + LocatedBlocks.class); + } + + @Override + public FsServerDefaults getServerDefaults() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getServerDefaults(); + } + GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST; + + return asyncIpcClient(() -> rpcProxy.getServerDefaults(null, req), + res -> PBHelperClient.convert(res.getServerDefaults()), + 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 { + 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(); + + return asyncIpcClient(() -> rpcProxy.create(null, req), + res -> res.hasFs() ? PBHelperClient.convert(res.getFs()) : null, + HdfsFileStatus.class); + } + + @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(); + + return asyncIpcClient(() -> rpcProxy.truncate(null, req), + res -> res.getResult(), Boolean.class); + } + + @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(); + + return asyncIpcClient(() -> rpcProxy.append(null, req), + res -> { + LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient + .convertLocatedBlockProto(res.getBlock()) : null; + HdfsFileStatus stat = (res.hasStat()) ? + PBHelperClient.convert(res.getStat()) : null; + return new LastBlockWithStatus(lastBlock, stat); + }, LastBlockWithStatus.class); + } + + @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(); + + return asyncIpcClient(() -> rpcProxy.setReplication(null, req), + res -> res.getResult(), Boolean.class); + } + + @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(); + + asyncIpcClient(() -> rpcProxy.setPermission(null, req), + res -> null, null); + } + + @Override + public void setOwner(String src, String username, String groupname) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setOwner(src, username, groupname); + return; + } + SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder() + .setSrc(src); + if (username != null) { + req.setUsername(username); + } + if (groupname != null) { + req.setGroupname(groupname); + } + + asyncIpcClient(() -> rpcProxy.setOwner(null, req.build()), + res -> null, null); + } + + @Override + public void abandonBlock(ExtendedBlock b, long fileId, String src, + String holder) throws IOException { + if (!Client.isAsynchronousMode()) { + super.abandonBlock(b, fileId, src, holder); + return; + } + AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder() + .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder) + .setFileId(fileId).build(); + asyncIpcClient(() -> rpcProxy.abandonBlock(null, req), + res -> null, 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)); + } + + return asyncIpcClient(() -> rpcProxy.addBlock(null, req.build()), + res -> PBHelperClient.convertLocatedBlockProto(res.getBlock()), + LocatedBlock.class); + } + + @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(); + + return asyncIpcClient(() -> rpcProxy.getAdditionalDatanode(null, req), + res -> PBHelperClient.convertLocatedBlockProto(res.getBlock()), + LocatedBlock.class); + } + + @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)); + } + + return asyncIpcClient(() -> rpcProxy.complete(null, req.build()), + res -> res.getResult(), + Boolean.class); + } + + @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(); + + asyncIpcClient(() -> rpcProxy.reportBadBlocks(null, req), + res -> null, Void.class); + } + + @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(); + + return asyncIpcClient(() -> rpcProxy.rename(null, req), + res -> res.getResult(), + Boolean.class); + } + + + @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(); + + asyncIpcClient(() -> rpcProxy.rename2(null, req), + res -> null, Void.class); + } + + @Override + public void concat(String trg, String[] srcs) throws IOException { + if (!Client.isAsynchronousMode()) { + super.concat(trg, srcs); + return; + } + ConcatRequestProto req = ConcatRequestProto.newBuilder(). + setTrg(trg). + addAllSrcs(Arrays.asList(srcs)).build(); + + asyncIpcClient(() -> rpcProxy.concat(null, req), + res -> null, Void.class); + } + + + @Override + public boolean delete(String src, boolean recursive) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.delete(src, recursive); + } + DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src) + .setRecursive(recursive).build(); + + return asyncIpcClient(() -> rpcProxy.delete(null, req), + res -> res.getResult(), Boolean.class); + } + + @Override + public boolean mkdirs(String src, FsPermission masked, boolean createParent) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.mkdirs(src, masked, createParent); + } + 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(); + + return asyncIpcClient(() -> rpcProxy.mkdirs(null, req), + res -> res.getResult(), Boolean.class); + } + + @Override + public DirectoryListing getListing( + String src, byte[] startAfter, boolean needLocation) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getListing(src, startAfter, needLocation); + } + GetListingRequestProto req = GetListingRequestProto.newBuilder() + .setSrc(src) + .setStartAfter(ByteString.copyFrom(startAfter)) + .setNeedLocation(needLocation).build(); + + return asyncIpcClient(() -> rpcProxy.getListing(null, req), + res -> { + if (res.hasDirList()) { + return PBHelperClient.convert(res.getDirList()); + } + return null; + }, DirectoryListing.class); + } + + @Override + public BatchedDirectoryListing getBatchedListing( + String[] srcs, byte[] startAfter, boolean needLocation) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBatchedListing(srcs, startAfter, needLocation); + } + GetBatchedListingRequestProto req = GetBatchedListingRequestProto + .newBuilder() + .addAllPaths(Arrays.asList(srcs)) + .setStartAfter(ByteString.copyFrom(startAfter)) + .setNeedLocation(needLocation).build(); + + return asyncIpcClient(() -> rpcProxy.getBatchedListing(null, req), + res -> { + if (res.getListingsCount() > 0) { + HdfsPartialListing[] listingArray = + new HdfsPartialListing[res.getListingsCount()]; + int listingIdx = 0; + for (BatchedDirectoryListingProto proto : res.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, res.getHasMore(), + res.getStartAfter().toByteArray()); + return batchedListing; + } + return null; + }, BatchedDirectoryListing.class); + } + + + @Override + public void renewLease(String clientName, List namespaces) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.renewLease(clientName, namespaces); + return; + } + RenewLeaseRequestProto.Builder builder = RenewLeaseRequestProto + .newBuilder().setClientName(clientName); + if (namespaces != null && !namespaces.isEmpty()) { + builder.addAllNamespaces(namespaces); + } + + asyncIpcClient(() -> rpcProxy.renewLease(null, builder.build()), + res -> null, Void.class); + } + + @Override + public boolean recoverLease(String src, String clientName) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.recoverLease(src, clientName); + } + RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder() + .setSrc(src) + .setClientName(clientName).build(); + + return asyncIpcClient(() -> rpcProxy.recoverLease(null, req), + res -> res.getResult(), Boolean.class); + } + + @Override + public long[] getStats() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getStats(); + } + + return asyncIpcClient(() -> rpcProxy.getFsStats(null, VOID_GET_FSSTATUS_REQUEST), + res -> PBHelperClient.convert(res), long[].class); + } + + @Override + public ReplicatedBlockStats getReplicatedBlockStats() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getReplicatedBlockStats(); + } + + return asyncIpcClient(() -> rpcProxy.getFsReplicatedBlockStats(null, + VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST), + res -> PBHelperClient.convert(res), ReplicatedBlockStats.class); + } + + @Override + public ECBlockGroupStats getECBlockGroupStats() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getECBlockGroupStats(); + } + + return asyncIpcClient(() -> rpcProxy.getFsECBlockGroupStats(null, + VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST), + res -> PBHelperClient.convert(res), ECBlockGroupStats.class); + } + + @Override + public DatanodeInfo[] getDatanodeReport(DatanodeReportType type) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getDatanodeReport(type); + } + GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto + .newBuilder() + .setType(PBHelperClient.convert(type)).build(); + + return asyncIpcClient(() -> rpcProxy.getDatanodeReport(null, req), + res -> PBHelperClient.convert(res.getDiList()), DatanodeInfo[].class); + } + + @Override + public DatanodeStorageReport[] getDatanodeStorageReport( + DatanodeReportType type) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getDatanodeStorageReport(type); + } + final GetDatanodeStorageReportRequestProto req + = GetDatanodeStorageReportRequestProto.newBuilder() + .setType(PBHelperClient.convert(type)).build(); + + return asyncIpcClient(() -> rpcProxy.getDatanodeStorageReport(null, req), + res -> PBHelperClient.convertDatanodeStorageReports( + res.getDatanodeStorageReportsList()), DatanodeStorageReport[].class); + } + + @Override + public long getPreferredBlockSize(String filename) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getPreferredBlockSize(filename); + } + GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto + .newBuilder() + .setFilename(filename) + .build(); + + return asyncIpcClient(() -> rpcProxy.getPreferredBlockSize(null, req), + res -> res.getBsize(), Long.class); + } + + @Override + public boolean setSafeMode(SafeModeAction action, boolean isChecked) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.setSafeMode(action, isChecked); + } + SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder() + .setAction(PBHelperClient.convert(action)) + .setChecked(isChecked).build(); + + return asyncIpcClient(() -> rpcProxy.setSafeMode(null, req), + res -> res.getResult(), Boolean.class); + } + + @Override + public boolean saveNamespace(long timeWindow, long txGap) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.saveNamespace(timeWindow, txGap); + } + SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder() + .setTimeWindow(timeWindow).setTxGap(txGap).build(); + + return asyncIpcClient(() -> rpcProxy.saveNamespace(null, req), + res -> res.getSaved(), Boolean.class); + } + + @Override + public long rollEdits() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.rollEdits(); + } + return asyncIpcClient(() -> rpcProxy.rollEdits(null, VOID_ROLLEDITS_REQUEST), + res -> res.getNewSegmentTxId(), Long.class); + } + + @Override + public boolean restoreFailedStorage(String arg) throws IOException{ + if (!Client.isAsynchronousMode()) { + return super.restoreFailedStorage(arg); + } + RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto + .newBuilder() + .setArg(arg).build(); + + return asyncIpcClient(() -> rpcProxy.restoreFailedStorage(null, req), + res -> res.getResult(), Boolean.class); + } + + @Override + public void refreshNodes() throws IOException { + if (!Client.isAsynchronousMode()) { + super.refreshNodes(); + return; + } + asyncIpcClient(() -> rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST), + res -> null, Void.class); + } + + @Override + public void finalizeUpgrade() throws IOException { + if (!Client.isAsynchronousMode()) { + super.finalizeUpgrade(); + return; + } + asyncIpcClient(() -> rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST), + res -> null, Void.class); + } + + @Override + public boolean upgradeStatus() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.upgradeStatus(); + } + return asyncIpcClient(() -> rpcProxy.upgradeStatus(null, VOID_UPGRADE_STATUS_REQUEST), + res -> res.getUpgradeFinalized(), Boolean.class); + } + + @Override + public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.rollingUpgrade(action); + } + final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder() + .setAction(PBHelperClient.convert(action)).build(); + + return asyncIpcClient(() -> rpcProxy.rollingUpgrade(null, r), + res -> PBHelperClient.convert(res.getRollingUpgradeInfo()), + RollingUpgradeInfo.class); + } + + @Override + public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.listCorruptFileBlocks(path, cookie); + } + ListCorruptFileBlocksRequestProto.Builder req = + ListCorruptFileBlocksRequestProto.newBuilder().setPath(path); + if (cookie != null) { + req.setCookie(cookie); + } + + return asyncIpcClient(() -> rpcProxy.listCorruptFileBlocks(null, req.build()), + res ->PBHelperClient.convert(res.getCorrupt()), CorruptFileBlocks.class); + } + + @Override + public void metaSave(String filename) throws IOException { + if (!Client.isAsynchronousMode()) { + super.metaSave(filename); + return; + } + MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder() + .setFilename(filename).build(); + + asyncIpcClient(() -> rpcProxy.metaSave(null, req), + res -> null, Void.class); + } + + @Override + public HdfsFileStatus getFileInfo(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getFileInfo(src); + } + GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder() + .setSrc(src) + .build(); + + return asyncIpcClient(() -> rpcProxy.getFileInfo(null, req), + res -> res.hasFs() ? PBHelperClient.convert(res.getFs()) : null, + HdfsFileStatus.class); + } + + @Override + public HdfsLocatedFileStatus getLocatedFileInfo( + String src, boolean needBlockToken) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getLocatedFileInfo(src, needBlockToken); + } + GetLocatedFileInfoRequestProto req = + GetLocatedFileInfoRequestProto.newBuilder() + .setSrc(src) + .setNeedBlockToken(needBlockToken) + .build(); + + return asyncIpcClient(() -> rpcProxy.getLocatedFileInfo(null, req), + res -> (HdfsLocatedFileStatus) (res.hasFs() ? PBHelperClient.convert(res.getFs()) : null), + HdfsLocatedFileStatus.class); + } + + @Override + public HdfsFileStatus getFileLinkInfo(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getFileLinkInfo(src); + } + GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder() + .setSrc(src).build(); + + return asyncIpcClient(() -> rpcProxy.getFileLinkInfo(null, req), + res -> res.hasFs() ? PBHelperClient.convert(res.getFs()) : null, + HdfsFileStatus.class); + } + + @Override + public ContentSummary getContentSummary(String path) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getContentSummary(path); + } + GetContentSummaryRequestProto req = GetContentSummaryRequestProto + .newBuilder() + .setPath(path) + .build(); + + return asyncIpcClient(() -> rpcProxy.getContentSummary(null, req), + res -> PBHelperClient.convert(res.getSummary()), ContentSummary.class); + } + + @Override + public void setQuota( + String path, long namespaceQuota, long storagespaceQuota, + StorageType type) throws IOException { + if (!Client.isAsynchronousMode()) { + super.setQuota(path, namespaceQuota, storagespaceQuota, type); + return; + } + 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(); + + asyncIpcClient(() -> rpcProxy.setQuota(null, req), + res -> null, Void.class); + } + + @Override + public void fsync( + String src, long fileId, String client, long lastBlockLength) throws IOException { + if (!Client.isAsynchronousMode()) { + super.fsync(src, fileId, client, lastBlockLength); + return; + } + FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src) + .setClient(client).setLastBlockLength(lastBlockLength) + .setFileId(fileId).build(); + + asyncIpcClient(() -> rpcProxy.fsync(null, req), + res -> null, Void.class); + } + + @Override + public void setTimes(String src, long mtime, long atime) throws IOException { + if (!Client.isAsynchronousMode()) { + super.setTimes(src, mtime, atime); + return; + } + SetTimesRequestProto req = SetTimesRequestProto.newBuilder() + .setSrc(src) + .setMtime(mtime) + .setAtime(atime) + .build(); + + asyncIpcClient(() -> rpcProxy.setTimes(null, req), + res -> null, Void.class); + } + + @Override + public void createSymlink( + String target, String link, FsPermission dirPerm, + boolean createParent) throws IOException { + if (!Client.isAsynchronousMode()) { + super.createSymlink(target, link, dirPerm, createParent); + return; + } + CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder() + .setTarget(target) + .setLink(link) + .setDirPerm(PBHelperClient.convert(dirPerm)) + .setCreateParent(createParent) + .build(); + + asyncIpcClient(() -> rpcProxy.createSymlink(null, req), + res -> null, Void.class); + } + + @Override + public String getLinkTarget(String path) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getLinkTarget(path); + } + GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder() + .setPath(path).build(); + + return asyncIpcClient(() -> rpcProxy.getLinkTarget(null, req), + res -> res.hasTargetPath() ? res.getTargetPath() : null, + String.class); + } + + @Override + public LocatedBlock updateBlockForPipeline( + ExtendedBlock block, String clientName) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.updateBlockForPipeline(block, clientName); + } + UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto + .newBuilder() + .setBlock(PBHelperClient.convert(block)) + .setClientName(clientName) + .build(); + + return asyncIpcClient(() -> rpcProxy.updateBlockForPipeline(null, req), + res -> PBHelperClient.convertLocatedBlockProto(res.getBlock()), + LocatedBlock.class); + } + + @Override + public void updatePipeline( + String clientName, ExtendedBlock oldBlock, ExtendedBlock newBlock, + DatanodeID[] newNodes, String[] storageIDs) throws IOException { + if (!Client.isAsynchronousMode()) { + super.updatePipeline(clientName, oldBlock, newBlock, newNodes, storageIDs); + return; + } + 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(); + + asyncIpcClient(() -> rpcProxy.updatePipeline(null, req), + res -> null, Void.class); + } + + @Override + public Token getDelegationToken( + Text renewer) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getDelegationToken(renewer); + } + GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto + .newBuilder() + .setRenewer(renewer == null ? "" : renewer.toString()) + .build(); + + return asyncIpcClient(() -> rpcProxy.getDelegationToken(null, req), + res -> res.hasToken() ? + PBHelperClient.convertDelegationToken(res.getToken()) : null, Token.class); + } + + @Override + public long renewDelegationToken( + Token token) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.renewDelegationToken(token); + } + RenewDelegationTokenRequestProto req = + RenewDelegationTokenRequestProto.newBuilder(). + setToken(PBHelperClient.convert(token)). + build(); + + return asyncIpcClient(() -> rpcProxy.renewDelegationToken(null, req), + res -> res.getNewExpiryTime(), Long.class); + } + + @Override + public void cancelDelegationToken( + Token token) throws IOException { + if (!Client.isAsynchronousMode()) { + super.cancelDelegationToken(token); + return; + } + CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto + .newBuilder() + .setToken(PBHelperClient.convert(token)) + .build(); + + asyncIpcClient(() -> rpcProxy.cancelDelegationToken(null, req), + res -> null, Void.class); + } + + @Override + public void setBalancerBandwidth(long bandwidth) throws IOException { + if (!Client.isAsynchronousMode()) { + super.setBalancerBandwidth(bandwidth); + return; + } + SetBalancerBandwidthRequestProto req = + SetBalancerBandwidthRequestProto.newBuilder() + .setBandwidth(bandwidth) + .build(); + + asyncIpcClient(() -> rpcProxy.setBalancerBandwidth(null, req), + res -> null, Void.class); + } + + @Override + public DataEncryptionKey getDataEncryptionKey() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getDataEncryptionKey(); + } + return asyncIpcClient(() -> rpcProxy.getDataEncryptionKey(null, + VOID_GET_DATA_ENCRYPTIONKEY_REQUEST), + res -> res.hasDataEncryptionKey() ? + PBHelperClient.convert(res.getDataEncryptionKey()) : null, + DataEncryptionKey.class); + } + + + @Override + public boolean isFileClosed(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.isFileClosed(src); + } + IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder() + .setSrc(src).build(); + + return asyncIpcClient(() -> rpcProxy.isFileClosed(null, req), + res -> res.getResult(), Boolean.class); + } + + @Override + public String createSnapshot( + String snapshotRoot, String snapshotName) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.createSnapshot(snapshotRoot, snapshotName); + } + final CreateSnapshotRequestProto.Builder builder + = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot); + if (snapshotName != null) { + builder.setSnapshotName(snapshotName); + } + final CreateSnapshotRequestProto req = builder.build(); + + return asyncIpcClient(() -> rpcProxy.createSnapshot(null, req), + res -> res.getSnapshotPath(), String.class); + } + + @Override + public void deleteSnapshot(String snapshotRoot, String snapshotName) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.deleteSnapshot(snapshotRoot, snapshotName); + return; + } + DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build(); + + asyncIpcClient(() -> rpcProxy.deleteSnapshot(null, req), + res -> null, Void.class); + } + + @Override + public void allowSnapshot(String snapshotRoot) throws IOException { + if (!Client.isAsynchronousMode()) { + super.allowSnapshot(snapshotRoot); + return; + } + AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).build(); + + asyncIpcClient(() -> rpcProxy.allowSnapshot(null, req), + res -> null, Void.class); + } + + @Override + public void disallowSnapshot(String snapshotRoot) throws IOException { + if (!Client.isAsynchronousMode()) { + super.disallowSnapshot(snapshotRoot); + return; + } + DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto + .newBuilder().setSnapshotRoot(snapshotRoot).build(); + + asyncIpcClient(() -> rpcProxy.disallowSnapshot(null, req), + res -> null, Void.class); + } + + @Override + public void renameSnapshot( + String snapshotRoot, String snapshotOldName, + String snapshotNewName) throws IOException { + if (!Client.isAsynchronousMode()) { + super.renameSnapshot(snapshotRoot, snapshotOldName, snapshotNewName); + return; + } + RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName) + .setSnapshotNewName(snapshotNewName).build(); + + asyncIpcClient(() -> rpcProxy.renameSnapshot(null, req), + res -> null, Void.class); + } + + @Override + public SnapshottableDirectoryStatus[] getSnapshottableDirListing() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getSnapshottableDirListing(); + } + GetSnapshottableDirListingRequestProto req = + GetSnapshottableDirListingRequestProto.newBuilder().build(); + + return asyncIpcClient(() -> rpcProxy.getSnapshottableDirListing(null, req), + res -> { + if (res.hasSnapshottableDirList()) { + return PBHelperClient.convert(res.getSnapshottableDirList()); + } + return null; + }, SnapshottableDirectoryStatus[].class); + } + + @Override + public SnapshotStatus[] getSnapshotListing(String path) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getSnapshotListing(path); + } + GetSnapshotListingRequestProto req = + GetSnapshotListingRequestProto.newBuilder() + .setSnapshotRoot(path).build(); + + return asyncIpcClient(() -> rpcProxy.getSnapshotListing(null, req), + res -> { + if (res.hasSnapshotList()) { + return PBHelperClient.convert(res.getSnapshotList()); + } + return null; + }, SnapshotStatus[].class); + } + + @Override + public SnapshotDiffReport getSnapshotDiffReport( + String snapshotRoot, String fromSnapshot, String toSnapshot) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getSnapshotDiffReport(snapshotRoot, fromSnapshot, toSnapshot); + } + GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto + .newBuilder().setSnapshotRoot(snapshotRoot) + .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build(); + + return asyncIpcClient(() -> rpcProxy.getSnapshotDiffReport(null, req), + res -> PBHelperClient.convert(res.getDiffReport()), SnapshotDiffReport.class); + } + + @Override + public SnapshotDiffReportListing getSnapshotDiffReportListing( + String snapshotRoot, String fromSnapshot, String toSnapshot, + byte[] startPath, int index) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getSnapshotDiffReportListing(snapshotRoot, fromSnapshot, + toSnapshot, startPath, index); + } + GetSnapshotDiffReportListingRequestProto req = + GetSnapshotDiffReportListingRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setFromSnapshot(fromSnapshot) + .setToSnapshot(toSnapshot).setCursor( + HdfsProtos.SnapshotDiffReportCursorProto.newBuilder() + .setStartPath(PBHelperClient.getByteString(startPath)) + .setIndex(index).build()).build(); + + return asyncIpcClient(() -> rpcProxy.getSnapshotDiffReportListing(null, req), + res -> PBHelperClient.convert(res.getDiffReport()), + SnapshotDiffReportListing.class); + } + + @Override + public long addCacheDirective( + CacheDirectiveInfo directive, EnumSet flags) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.addCacheDirective(directive, flags); + } + AddCacheDirectiveRequestProto.Builder builder = + AddCacheDirectiveRequestProto.newBuilder(). + setInfo(PBHelperClient.convert(directive)); + if (!flags.isEmpty()) { + builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags)); + } + + return asyncIpcClient(() -> rpcProxy.addCacheDirective(null, builder.build()), + res -> res.getId(), Long.class); + } + + @Override + public void modifyCacheDirective( + CacheDirectiveInfo directive, EnumSet flags) throws IOException { + if (!Client.isAsynchronousMode()) { + super.modifyCacheDirective(directive, flags); + return; + } + ModifyCacheDirectiveRequestProto.Builder builder = + ModifyCacheDirectiveRequestProto.newBuilder(). + setInfo(PBHelperClient.convert(directive)); + if (!flags.isEmpty()) { + builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags)); + } + + asyncIpcClient(() -> rpcProxy.modifyCacheDirective(null, builder.build()), + res -> null, Void.class); + } + + @Override + public void removeCacheDirective(long id) throws IOException { + if (!Client.isAsynchronousMode()) { + super.removeCacheDirective(id); + return; + } + + asyncIpcClient(() -> rpcProxy.removeCacheDirective(null, + RemoveCacheDirectiveRequestProto.newBuilder(). + setId(id).build()), + res -> null, Void.class); + } + + @Override + public BatchedEntries listCacheDirectives( + long prevId, CacheDirectiveInfo filter) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.listCacheDirectives(prevId, filter); + } + if (filter == null) { + filter = new CacheDirectiveInfo.Builder().build(); + } + CacheDirectiveInfo f = filter; + + return asyncIpcClient(() -> rpcProxy.listCacheDirectives(null, + ListCacheDirectivesRequestProto.newBuilder(). + setPrevId(prevId). + setFilter(PBHelperClient.convert(f)). + build()), + res -> new BatchedCacheEntries(res), BatchedEntries.class); + } + + @Override + public void addCachePool(CachePoolInfo info) throws IOException { + if (!Client.isAsynchronousMode()) { + super.addCachePool(info); + return; + } + AddCachePoolRequestProto.Builder builder = + AddCachePoolRequestProto.newBuilder(); + builder.setInfo(PBHelperClient.convert(info)); + + asyncIpcClient(() -> rpcProxy.addCachePool(null, builder.build()), + res -> null, Void.class); + } + + @Override + public void modifyCachePool(CachePoolInfo req) throws IOException { + if (!Client.isAsynchronousMode()) { + super.modifyCachePool(req); + return; + } + ModifyCachePoolRequestProto.Builder builder = + ModifyCachePoolRequestProto.newBuilder(); + builder.setInfo(PBHelperClient.convert(req)); + + asyncIpcClient(() -> rpcProxy.modifyCachePool(null, builder.build()), + res -> null, Void.class); + } + + @Override + public void removeCachePool(String cachePoolName) throws IOException { + if (!Client.isAsynchronousMode()) { + super.removeCachePool(cachePoolName); + return; + } + + asyncIpcClient(() -> rpcProxy.removeCachePool(null, + RemoveCachePoolRequestProto.newBuilder(). + setPoolName(cachePoolName).build()), + res -> null, Void.class); + } + + @Override + public BatchedEntries listCachePools(String prevKey) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.listCachePools(prevKey); + } + + return asyncIpcClient(() -> rpcProxy.listCachePools(null, + ListCachePoolsRequestProto.newBuilder().setPrevPoolName(prevKey).build()), + res -> new BatchedCachePoolEntries(res), BatchedEntries.class); + } + + @Override + public void modifyAclEntries(String src, List aclSpec) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.modifyAclEntries(src, aclSpec); + return; + } + ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto + .newBuilder().setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build(); + + asyncIpcClient(() -> rpcProxy.modifyAclEntries(null, req), + res -> null, Void.class); + } + + @Override + public void removeAclEntries(String src, List aclSpec) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.removeAclEntries(src, aclSpec); + return; + } + RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto + .newBuilder().setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build(); + + asyncIpcClient(() -> rpcProxy.removeAclEntries(null, req), + res -> null, Void.class); + } + + @Override + public void removeDefaultAcl(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + super.removeDefaultAcl(src); + return; + } + RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto + .newBuilder().setSrc(src).build(); + + asyncIpcClient(() -> rpcProxy.removeDefaultAcl(null, req), + res -> null, Void.class); + } + + @Override + public void removeAcl(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + super.removeAcl(src); + return; + } + RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder() + .setSrc(src).build(); + + asyncIpcClient(() -> rpcProxy.removeAcl(null, req), + res -> null, Void.class); + } + + @Override + public void setAcl(String src, List aclSpec) throws IOException { + if (!Client.isAsynchronousMode()) { + super.setAcl(src, aclSpec); + return; + } + SetAclRequestProto req = SetAclRequestProto.newBuilder() + .setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)) + .build(); + + asyncIpcClient(() -> rpcProxy.setAcl(null, req), + res -> null, Void.class); + } + + @Override + public AclStatus getAclStatus(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getAclStatus(src); + } + GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder() + .setSrc(src).build(); + + return asyncIpcClient(() -> rpcProxy.getAclStatus(null, req), + res -> PBHelperClient.convert(res), AclStatus.class); + } + + @Override + public void createEncryptionZone(String src, String keyName) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.createEncryptionZone(src, keyName); + return; + } + final CreateEncryptionZoneRequestProto.Builder builder = + CreateEncryptionZoneRequestProto.newBuilder(); + builder.setSrc(src); + if (keyName != null && !keyName.isEmpty()) { + builder.setKeyName(keyName); + } + CreateEncryptionZoneRequestProto req = builder.build(); + + asyncIpcClient(() -> rpcProxy.createEncryptionZone(null, req), + res -> null, Void.class); + } + + @Override + public EncryptionZone getEZForPath(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getEZForPath(src); + } + final GetEZForPathRequestProto.Builder builder = + GetEZForPathRequestProto.newBuilder(); + builder.setSrc(src); + final GetEZForPathRequestProto req = builder.build(); + + return asyncIpcClient(() -> rpcProxy.getEZForPath(null, req), + res -> { + if (res.hasZone()) { + return PBHelperClient.convert(res.getZone()); + } else { + return null; + } + }, EncryptionZone.class); + } + + @Override + public BatchedEntries listEncryptionZones(long id) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.listEncryptionZones(id); + } + final ListEncryptionZonesRequestProto req = + ListEncryptionZonesRequestProto.newBuilder() + .setId(id) + .build(); + + return asyncIpcClient(() -> rpcProxy.listEncryptionZones(null, req), + res -> { + List elements = + Lists.newArrayListWithCapacity(res.getZonesCount()); + for (EncryptionZoneProto p : res.getZonesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, res.getHasMore()); + }, BatchedEntries.class); + } + + @Override + public void setErasureCodingPolicy(String src, String ecPolicyName) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setErasureCodingPolicy(src, ecPolicyName); + return; + } + final SetErasureCodingPolicyRequestProto.Builder builder = + SetErasureCodingPolicyRequestProto.newBuilder(); + builder.setSrc(src); + if (ecPolicyName != null) { + builder.setEcPolicyName(ecPolicyName); + } + SetErasureCodingPolicyRequestProto req = builder.build(); + + asyncIpcClient(() -> rpcProxy.setErasureCodingPolicy(null, req), + res -> null, Void.class); + } + + @Override + public void unsetErasureCodingPolicy(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + super.unsetErasureCodingPolicy(src); + return; + } + final UnsetErasureCodingPolicyRequestProto.Builder builder = + UnsetErasureCodingPolicyRequestProto.newBuilder(); + builder.setSrc(src); + UnsetErasureCodingPolicyRequestProto req = builder.build(); + + asyncIpcClient(() -> rpcProxy.unsetErasureCodingPolicy(null, req), + res -> null, Void.class); + } + + @Override + public ECTopologyVerifierResult getECTopologyResultForPolicies( + final String... policyNames) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getECTopologyResultForPolicies(policyNames); + } + final GetECTopologyResultForPoliciesRequestProto.Builder builder = + GetECTopologyResultForPoliciesRequestProto.newBuilder(); + builder.addAllPolicies(Arrays.asList(policyNames)); + GetECTopologyResultForPoliciesRequestProto req = builder.build(); + + return asyncIpcClient(() -> rpcProxy.getECTopologyResultForPolicies(null, req), + res -> PBHelperClient.convertECTopologyVerifierResultProto(res.getResponse()), + ECTopologyVerifierResult.class); + } + + @Override + public void reencryptEncryptionZone(String zone, ReencryptAction action) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.reencryptEncryptionZone(zone, action); + return; + } + final ReencryptEncryptionZoneRequestProto.Builder builder = + ReencryptEncryptionZoneRequestProto.newBuilder(); + builder.setZone(zone).setAction(PBHelperClient.convert(action)); + ReencryptEncryptionZoneRequestProto req = builder.build(); + + asyncIpcClient(() -> rpcProxy.reencryptEncryptionZone(null, req), + res -> null, Void.class); + } + + @Override + public BatchedEntries listReencryptionStatus(long id) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.listReencryptionStatus(id); + } + final ListReencryptionStatusRequestProto req = + ListReencryptionStatusRequestProto.newBuilder().setId(id).build(); + + return asyncIpcClient(() -> rpcProxy.listReencryptionStatus(null, req), + res -> { + List elements = + Lists.newArrayListWithCapacity(res.getStatusesCount()); + for (ZoneReencryptionStatusProto p : res.getStatusesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, res.getHasMore()); + }, BatchedEntries.class); + } + + @Override + public void setXAttr(String src, XAttr xAttr, EnumSet flag) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setXAttr(src, xAttr, flag); + return; + } + SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder() + .setSrc(src) + .setXAttr(PBHelperClient.convertXAttrProto(xAttr)) + .setFlag(PBHelperClient.convert(flag)) + .build(); + + asyncIpcClient(() -> rpcProxy.setXAttr(null, req), + res -> null, Void.class); + } + + @Override + public List getXAttrs(String src, List xAttrs) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getXAttrs(src, xAttrs); + } + GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder(); + builder.setSrc(src); + if (xAttrs != null) { + builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs)); + } + GetXAttrsRequestProto req = builder.build(); + + return asyncIpcClient(() -> rpcProxy.getXAttrs(null, req), + res -> PBHelperClient.convert(res), List.class); + } + + @Override + public List listXAttrs(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.listXAttrs(src); + } + ListXAttrsRequestProto.Builder builder = + ListXAttrsRequestProto.newBuilder(); + builder.setSrc(src); + ListXAttrsRequestProto req = builder.build(); + + return asyncIpcClient(() -> rpcProxy.listXAttrs(null, req), + res -> PBHelperClient.convert(res), List.class); + } + + @Override + public void removeXAttr(String src, XAttr xAttr) throws IOException { + if (!Client.isAsynchronousMode()) { + super.removeXAttr(src, xAttr); + return; + } + RemoveXAttrRequestProto req = RemoveXAttrRequestProto + .newBuilder().setSrc(src) + .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build(); + + asyncIpcClient(() -> rpcProxy.removeXAttr(null, req), + res -> null, Void.class); + } + + @Override + public void checkAccess(String path, FsAction mode) throws IOException { + if (!Client.isAsynchronousMode()) { + super.checkAccess(path, mode); + return; + } + CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder() + .setPath(path).setMode(PBHelperClient.convert(mode)).build(); + + asyncIpcClient(() -> rpcProxy.checkAccess(null, req), + res -> null, Void.class); + } + + @Override + public void setStoragePolicy(String src, String policyName) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setStoragePolicy(src, policyName); + return; + } + SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto + .newBuilder().setSrc(src).setPolicyName(policyName).build(); + + asyncIpcClient(() -> rpcProxy.setStoragePolicy(null, req), + res -> null, Void.class); + } + + @Override + public void unsetStoragePolicy(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + super.unsetStoragePolicy(src); + return; + } + UnsetStoragePolicyRequestProto req = UnsetStoragePolicyRequestProto + .newBuilder().setSrc(src).build(); + + asyncIpcClient(() -> rpcProxy.unsetStoragePolicy(null, req), + res -> null, Void.class); + } + + @Override + public BlockStoragePolicy getStoragePolicy(String path) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getStoragePolicy(path); + } + GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto + .newBuilder().setPath(path).build(); + + return asyncIpcClient(() -> rpcProxy.getStoragePolicy(null, request), + res -> PBHelperClient.convert(res.getStoragePolicy()), + BlockStoragePolicy.class); + } + + @Override + public BlockStoragePolicy[] getStoragePolicies() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getStoragePolicies(); + } + + return asyncIpcClient(() -> rpcProxy.getStoragePolicies(null, + VOID_GET_STORAGE_POLICIES_REQUEST), + res -> PBHelperClient.convertStoragePolicies(res.getPoliciesList()), + BlockStoragePolicy[].class); + } + + public long getCurrentEditLogTxid() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getCurrentEditLogTxid(); + } + GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto + .getDefaultInstance(); + + return asyncIpcClient(() -> rpcProxy.getCurrentEditLogTxid(null, req), + res -> res.getTxid(), Long.class); + } + + @Override + public EventBatchList getEditsFromTxid(long txid) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getEditsFromTxid(txid); + } + GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder() + .setTxid(txid).build(); + + return asyncIpcClient(() -> rpcProxy.getEditsFromTxid(null, req), + res -> PBHelperClient.convert(res), EventBatchList.class); + } + + @Override + public AddErasureCodingPolicyResponse[] addErasureCodingPolicies( + ErasureCodingPolicy[] policies) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.addErasureCodingPolicies(policies); + } + List protos = Arrays.stream(policies) + .map(PBHelperClient::convertErasureCodingPolicy) + .collect(Collectors.toList()); + AddErasureCodingPoliciesRequestProto req = + AddErasureCodingPoliciesRequestProto.newBuilder() + .addAllEcPolicies(protos).build(); + + return asyncIpcClient(() -> rpcProxy.addErasureCodingPolicies(null, req), + res -> res.getResponsesList().stream() + .map(PBHelperClient::convertAddErasureCodingPolicyResponse) + .toArray(AddErasureCodingPolicyResponse[]::new), + AddErasureCodingPolicyResponse[].class); + } + + @Override + public void removeErasureCodingPolicy(String ecPolicyName) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.removeErasureCodingPolicy(ecPolicyName); + return; + } + RemoveErasureCodingPolicyRequestProto.Builder builder = + RemoveErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + RemoveErasureCodingPolicyRequestProto req = builder.build(); + + asyncIpcClient(() -> rpcProxy.removeErasureCodingPolicy(null, req), + res -> null, Void.class); + } + + @Override + public void enableErasureCodingPolicy(String ecPolicyName) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.enableErasureCodingPolicy(ecPolicyName); + return; + } + EnableErasureCodingPolicyRequestProto.Builder builder = + EnableErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + EnableErasureCodingPolicyRequestProto req = builder.build(); + + asyncIpcClient(() -> rpcProxy.enableErasureCodingPolicy(null, req), + res -> null, Void.class); + } + + @Override + public void disableErasureCodingPolicy(String ecPolicyName) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.disableErasureCodingPolicy(ecPolicyName); + return; + } + DisableErasureCodingPolicyRequestProto.Builder builder = + DisableErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + DisableErasureCodingPolicyRequestProto req = builder.build(); + + asyncIpcClient(() -> rpcProxy.disableErasureCodingPolicy(null, req), + res -> null, Void.class); + } + + @Override + public ErasureCodingPolicyInfo[] getErasureCodingPolicies() + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getErasureCodingPolicies(); + } + + return asyncIpcClient(() -> rpcProxy.getErasureCodingPolicies( + null, VOID_GET_EC_POLICIES_REQUEST), + res -> { + ErasureCodingPolicyInfo[] ecPolicies = + new ErasureCodingPolicyInfo[res.getEcPoliciesCount()]; + int i = 0; + for (ErasureCodingPolicyProto proto : res.getEcPoliciesList()) { + ecPolicies[i++] = + PBHelperClient.convertErasureCodingPolicyInfo(proto); + } + return ecPolicies; + }, ErasureCodingPolicyInfo[].class); + } + + @Override + public Map getErasureCodingCodecs() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getErasureCodingCodecs(); + } + + return asyncIpcClient(() -> rpcProxy + .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST), + res -> { + Map ecCodecs = new HashMap<>(); + for (CodecProto codec : res.getCodecList()) { + ecCodecs.put(codec.getCodec(), codec.getCoders()); + } + return ecCodecs; + }, Map.class); + } + + @Override + public ErasureCodingPolicy getErasureCodingPolicy(String src) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getErasureCodingPolicy(src); + } + GetErasureCodingPolicyRequestProto req = + GetErasureCodingPolicyRequestProto.newBuilder().setSrc(src).build(); + + return asyncIpcClient(() -> rpcProxy.getErasureCodingPolicy(null, req), + res -> { + if (res.hasEcPolicy()) { + return PBHelperClient.convertErasureCodingPolicy( + res.getEcPolicy()); + } + return null; + }, ErasureCodingPolicy.class); + } + + @Override + public QuotaUsage getQuotaUsage(String path) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getQuotaUsage(path); + } + GetQuotaUsageRequestProto req = + GetQuotaUsageRequestProto.newBuilder().setPath(path).build(); + + return asyncIpcClient(() -> rpcProxy.getQuotaUsage(null, req), + res -> PBHelperClient.convert(res.getUsage()), QuotaUsage.class); + } + + @Override + public BatchedEntries listOpenFiles( + long prevId, EnumSet openFilesTypes, + String path) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.listOpenFiles(prevId, openFilesTypes, path); + } + ListOpenFilesRequestProto.Builder req = + ListOpenFilesRequestProto.newBuilder().setId(prevId); + if (openFilesTypes != null) { + req.addAllTypes(PBHelperClient.convertOpenFileTypes(openFilesTypes)); + } + req.setPath(path); + + return asyncIpcClient(() -> rpcProxy.listOpenFiles(null, req.build()), + res -> { + List openFileEntries = + Lists.newArrayListWithCapacity(res.getEntriesCount()); + for (OpenFilesBatchResponseProto p : res.getEntriesList()) { + openFileEntries.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(openFileEntries, res.getHasMore()); + }, BatchedEntries.class); + } + + @Override + public void msync() throws IOException { + if (!Client.isAsynchronousMode()) { + super.msync(); + return; + } + MsyncRequestProto.Builder req = MsyncRequestProto.newBuilder(); + + asyncIpcClient(() -> rpcProxy.msync(null, req.build()), + res -> null, Void.class); + } + + @Override + public void satisfyStoragePolicy(String src) throws IOException { + if (!Client.isAsynchronousMode()) { + super.satisfyStoragePolicy(src); + return; + } + SatisfyStoragePolicyRequestProto req = + SatisfyStoragePolicyRequestProto.newBuilder().setSrc(src).build(); + + asyncIpcClient(() -> rpcProxy.satisfyStoragePolicy(null, req), + res -> null, Void.class); + } + + @Override + public DatanodeInfo[] getSlowDatanodeReport() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getSlowDatanodeReport(); + } + GetSlowDatanodeReportRequestProto req = + GetSlowDatanodeReportRequestProto.newBuilder().build(); + + return asyncIpcClient(() -> rpcProxy.getSlowDatanodeReport(null, req), + res -> PBHelperClient.convert(res.getDatanodeInfoProtoList()), + DatanodeInfo[].class); + } + + @Override + public HAServiceProtocol.HAServiceState getHAServiceState() + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getHAServiceState(); + } + HAServiceStateRequestProto req = + HAServiceStateRequestProto.newBuilder().build(); + + return asyncIpcClient(() -> rpcProxy.getHAServiceState(null, req), + res -> { + switch(res.getState()) { + 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; + } + }, HAServiceProtocol.HAServiceState.class); + } + + @Override + public Path getEnclosingRoot(String filename) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getEnclosingRoot(filename); + } + final GetEnclosingRootRequestProto.Builder builder = + GetEnclosingRootRequestProto.newBuilder(); + builder.setFilename(filename); + final GetEnclosingRootRequestProto req = builder.build(); + + return asyncIpcClient(() -> rpcProxy.getEnclosingRoot(null, req), + res -> new Path(res.getEnclosingRootPath()), + Path.class); + } +} 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..9f607f08f18b3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolServerSideTranslatorPB.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.thirdparty.protobuf.RpcController; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; +import org.apache.hadoop.tools.GetUserMappingsProtocol; +import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto; +import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto; +import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncRouterServer; + +public class RouterGetUserMappingsProtocolServerSideTranslatorPB + extends GetUserMappingsProtocolServerSideTranslatorPB { + private final RouterRpcServer server; + private final boolean isAsyncRpc; + + public RouterGetUserMappingsProtocolServerSideTranslatorPB(GetUserMappingsProtocol impl) { + super(impl); + this.server = (RouterRpcServer) impl; + this.isAsyncRpc = server.isAsync(); + } + + @Override + public GetGroupsForUserResponseProto getGroupsForUser( + RpcController controller, + GetGroupsForUserRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.getGroupsForUser(controller, request); + } + asyncRouterServer(() -> server.getGroupsForUser(request.getUser()), groups -> { + GetGroupsForUserResponseProto.Builder builder = + GetGroupsForUserResponseProto + .newBuilder(); + for (String g : groups) { + builder.addGroups(g); + } + return builder.build(); + }); + return null; + } +} 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..4d8dd7164ade0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolTranslatorPB.java @@ -0,0 +1,52 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos; +import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolClientSideTranslatorPB; +import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpcClient; + +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(); + + return asyncIpcClient(() -> rpcProxy.getGroupsForUser(null, request), + res -> res.getGroupsList().toArray(new String[res.getGroupsCount()]), + String[].class); + } +} 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..d706c1148e1b8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolServerSideTranslatorPB.java @@ -0,0 +1,276 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.EndCheckpointResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.ErrorReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlockKeysResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetBlocksResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetEditLogManifestResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentCheckpointTxIdResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentNameNodeFileTxIdRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentNameNodeFileTxIdResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetNextSPSPathResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsRollingUpgradeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.IsUpgradeFinalizedResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RegisterResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.StartCheckpointResponseProto; +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 GetBlocksResponseProto getBlocks(RpcController unused, + GetBlocksRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.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 -> + GetBlocksResponseProto.newBuilder() + .setBlocks(PBHelper.convert(blocks)).build()); + return null; + } + + @Override + public GetBlockKeysResponseProto getBlockKeys(RpcController unused, + GetBlockKeysRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.getBlockKeys(unused, request); + } + asyncRouterServer(server::getBlockKeys, keys -> { + GetBlockKeysResponseProto.Builder builder = + GetBlockKeysResponseProto.newBuilder(); + if (keys != null) { + builder.setKeys(PBHelper.convert(keys)); + } + return builder.build(); + }); + return null; + } + + @Override + public GetTransactionIdResponseProto getTransactionId(RpcController unused, + GetTransactionIdRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.getTransactionId(unused, request); + } + asyncRouterServer(server::getTransactionID, + txid -> GetTransactionIdResponseProto + .newBuilder().setTxId(txid).build()); + return null; + } + + @Override + public GetMostRecentCheckpointTxIdResponseProto getMostRecentCheckpointTxId( + RpcController unused, GetMostRecentCheckpointTxIdRequestProto request) + throws ServiceException { + if (!isAsyncRpc) { + return super.getMostRecentCheckpointTxId(unused, request); + } + asyncRouterServer(server::getMostRecentCheckpointTxId, + txid -> GetMostRecentCheckpointTxIdResponseProto + .newBuilder().setTxId(txid).build()); + return null; + } + + @Override + public GetMostRecentNameNodeFileTxIdResponseProto getMostRecentNameNodeFileTxId( + RpcController unused, GetMostRecentNameNodeFileTxIdRequestProto request) + throws ServiceException { + if (!isAsyncRpc) { + return super.getMostRecentNameNodeFileTxId(unused, request); + } + asyncRouterServer(() -> server.getMostRecentNameNodeFileTxId( + NNStorage.NameNodeFile.valueOf(request.getNameNodeFile())), + txid -> GetMostRecentNameNodeFileTxIdResponseProto + .newBuilder().setTxId(txid).build()); + return null; + } + + @Override + public RollEditLogResponseProto rollEditLog(RpcController unused, + RollEditLogRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.rollEditLog(unused, request); + } + asyncRouterServer(server::rollEditLog, + signature -> RollEditLogResponseProto.newBuilder() + .setSignature(PBHelper.convert(signature)).build()); + return null; + } + + @Override + public ErrorReportResponseProto errorReport(RpcController unused, + ErrorReportRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.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 RegisterResponseProto registerSubordinateNamenode( + RpcController unused, RegisterRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.registerSubordinateNamenode(unused, request); + } + asyncRouterServer(() -> server.registerSubordinateNamenode( + PBHelper.convert(request.getRegistration())), + reg -> RegisterResponseProto.newBuilder() + .setRegistration(PBHelper.convert(reg)).build()); + return null; + } + + @Override + public StartCheckpointResponseProto startCheckpoint(RpcController unused, + StartCheckpointRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.startCheckpoint(unused, request); + } + asyncRouterServer(() -> + server.startCheckpoint(PBHelper.convert(request.getRegistration())), + cmd -> StartCheckpointResponseProto.newBuilder() + .setCommand(PBHelper.convert(cmd)).build()); + return null; + } + + + @Override + public EndCheckpointResponseProto endCheckpoint(RpcController unused, + EndCheckpointRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.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 GetEditLogManifestResponseProto getEditLogManifest( + RpcController unused, GetEditLogManifestRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.getEditLogManifest(unused, request); + } + asyncRouterServer(() -> server.getEditLogManifest(request.getSinceTxId()), + manifest -> GetEditLogManifestResponseProto.newBuilder() + .setManifest(PBHelper.convert(manifest)).build()); + return null; + } + + @Override + public VersionResponseProto versionRequest( + RpcController controller, + VersionRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.versionRequest(controller, request); + } + asyncRouterServer(server::versionRequest, + info -> VersionResponseProto.newBuilder() + .setInfo(PBHelper.convert(info)).build()); + return null; + } + + @Override + public IsUpgradeFinalizedResponseProto isUpgradeFinalized(RpcController controller, + IsUpgradeFinalizedRequestProto request) throws ServiceException { + if (!isAsyncRpc) { + return super.isUpgradeFinalized(controller, request); + } + asyncRouterServer(server::isUpgradeFinalized, + isUpgradeFinalized -> IsUpgradeFinalizedResponseProto.newBuilder() + .setIsUpgradeFinalized(isUpgradeFinalized).build()); + return null; + } + + @Override + public IsRollingUpgradeResponseProto isRollingUpgrade( + RpcController controller, IsRollingUpgradeRequestProto request) + throws ServiceException { + if (!isAsyncRpc) { + return super.isRollingUpgrade(controller, request); + } + asyncRouterServer(server::isRollingUpgrade, + isRollingUpgrade -> IsRollingUpgradeResponseProto.newBuilder() + .setIsRollingUpgrade(isRollingUpgrade).build()); + return null; + } + + @Override + public GetNextSPSPathResponseProto getNextSPSPath( + RpcController controller, GetNextSPSPathRequestProto request) + throws ServiceException { + if (!isAsyncRpc) { + return super.getNextSPSPath(controller, request); + } + asyncRouterServer(server::getNextSPSPath, + nextSPSPath -> 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..54c099581423d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolTranslatorPB.java @@ -0,0 +1,270 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.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.GetBlockKeysRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetTransactionIdRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.RollEditLogRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.VersionRequestProto; +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 java.io.IOException; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpcClient; + +public class RouterNamenodeProtocolTranslatorPB extends NamenodeProtocolTranslatorPB { + /* + * Protobuf requests with no parameters instantiated only once + */ + private static final GetBlockKeysRequestProto VOID_GET_BLOCKKEYS_REQUEST = + GetBlockKeysRequestProto.newBuilder().build(); + private static final GetTransactionIdRequestProto VOID_GET_TRANSACTIONID_REQUEST = + GetTransactionIdRequestProto.newBuilder().build(); + private static final RollEditLogRequestProto VOID_ROLL_EDITLOG_REQUEST = + RollEditLogRequestProto.newBuilder().build(); + private static final VersionRequestProto VOID_VERSION_REQUEST = + VersionRequestProto.newBuilder().build(); + 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(); + + return asyncIpcClient(() -> rpcProxy.getBlocks(null, req), + res -> PBHelper.convert(res.getBlocks()), + BlocksWithLocations.class); + } + + @Override + public ExportedBlockKeys getBlockKeys() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBlockKeys(); + } + + return asyncIpcClient(() -> rpcProxy.getBlockKeys(null, + VOID_GET_BLOCKKEYS_REQUEST), + res -> res.hasKeys() ? PBHelper.convert(res.getKeys()) : null, + ExportedBlockKeys.class); + } + + @Override + public long getTransactionID() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getTransactionID(); + } + + return asyncIpcClient(() -> rpcProxy.getTransactionId(null, + VOID_GET_TRANSACTIONID_REQUEST), + res -> res.getTxId(), Long.class); + } + + @Override + public long getMostRecentCheckpointTxId() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getMostRecentCheckpointTxId(); + } + + return asyncIpcClient(() -> rpcProxy.getMostRecentCheckpointTxId(null, + NamenodeProtocolProtos + .GetMostRecentCheckpointTxIdRequestProto + .getDefaultInstance()), + res -> res.getTxId(), Long.class); + } + + @Override + public long getMostRecentNameNodeFileTxId(NNStorage.NameNodeFile nnf) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getMostRecentNameNodeFileTxId(nnf); + } + + return asyncIpcClient(() -> rpcProxy.getMostRecentNameNodeFileTxId(null, + NamenodeProtocolProtos + .GetMostRecentNameNodeFileTxIdRequestProto + .newBuilder() + .setNameNodeFile(nnf.toString()) + .build()), + res -> res.getTxId(), Long.class); + } + + @Override + public CheckpointSignature rollEditLog() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.rollEditLog(); + } + + return asyncIpcClient(() -> rpcProxy.rollEditLog(null, + VOID_ROLL_EDITLOG_REQUEST), + res -> PBHelper.convert(res.getSignature()), CheckpointSignature.class); + } + + @Override + public NamespaceInfo versionRequest() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.versionRequest(); + } + return asyncIpcClient(() -> rpcProxy.versionRequest(null, + VOID_VERSION_REQUEST), + res -> PBHelper.convert(res.getInfo()), + NamespaceInfo.class); + } + + @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(); + + asyncIpcClient(() -> rpcProxy.errorReport(null, req), + res -> null, Void.class); + } + + @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(); + + return asyncIpcClient(() -> rpcProxy.registerSubordinateNamenode(null, req), + res -> PBHelper.convert(res.getRegistration()), + NamenodeRegistration.class); + } + + @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(); + + return asyncIpcClient(() -> rpcProxy.startCheckpoint(null, req), + res -> { + HdfsServerProtos.NamenodeCommandProto cmd = res.getCommand(); + return PBHelper.convert(cmd); + }, NamenodeCommand.class); + } + + @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(); + + asyncIpcClient(() -> rpcProxy.endCheckpoint(null, req), + res -> null, Void.class); + } + + @Override + public RemoteEditLogManifest getEditLogManifest(long sinceTxId) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getEditLogManifest(sinceTxId); + } + NamenodeProtocolProtos.GetEditLogManifestRequestProto req = + NamenodeProtocolProtos.GetEditLogManifestRequestProto + .newBuilder().setSinceTxId(sinceTxId).build(); + + return asyncIpcClient(() -> rpcProxy.getEditLogManifest(null, req), + res -> PBHelper.convert(res.getManifest()), RemoteEditLogManifest.class); + } + + @Override + public boolean isUpgradeFinalized() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.isUpgradeFinalized(); + } + NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto req = + NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto + .newBuilder().build(); + + return asyncIpcClient(() -> rpcProxy.isUpgradeFinalized(null, req), + res -> res.getIsUpgradeFinalized(), Boolean.class); + } + + @Override + public boolean isRollingUpgrade() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.isRollingUpgrade(); + } + NamenodeProtocolProtos.IsRollingUpgradeRequestProto req = + NamenodeProtocolProtos.IsRollingUpgradeRequestProto + .newBuilder().build(); + + return asyncIpcClient(() -> rpcProxy.isRollingUpgrade(null, req), + res -> res.getIsRollingUpgrade(), Boolean.class); + } + + @Override + public Long getNextSPSPath() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getNextSPSPath(); + } + NamenodeProtocolProtos.GetNextSPSPathRequestProto req = + NamenodeProtocolProtos.GetNextSPSPathRequestProto.newBuilder().build(); + + return asyncIpcClient(() -> rpcProxy.getNextSPSPath(null, req), + res -> res.hasSpsPath() ? res.getSpsPath() : null, Long.class); + } +} 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..9cffac88ee60c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolServerSideTranslatorPB.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.protocolPB; + +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.security.RefreshUserMappingsProtocol; +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 org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolServerSideTranslatorPB; +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 RouterRefreshUserMappingsProtocolServerSideTranslatorPB + extends RefreshUserMappingsProtocolServerSideTranslatorPB { + + private final RouterRpcServer server; + private final boolean isAsyncRpc; + + private final static RefreshUserToGroupsMappingsResponseProto + VOID_REFRESH_USER_GROUPS_MAPPING_RESPONSE = + RefreshUserToGroupsMappingsResponseProto.newBuilder().build(); + + private final static RefreshSuperUserGroupsConfigurationResponseProto + VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_RESPONSE = + RefreshSuperUserGroupsConfigurationResponseProto.newBuilder() + .build(); + + 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..78728e04f295d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolTranslatorPB.java @@ -0,0 +1,64 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos; +import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB; +import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolPB; +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpcClient; + +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; + } + + asyncIpcClient(() -> rpcProxy.refreshUserToGroupsMappings(null, + RefreshUserMappingsProtocolProtos + .RefreshUserToGroupsMappingsRequestProto + .newBuilder().build()), + res -> null, Void.class); + } + + @Override + public void refreshSuperUserGroupsConfiguration() throws IOException { + if (!Client.isAsynchronousMode()) { + super.refreshSuperUserGroupsConfiguration(); + return; + } + + asyncIpcClient(() -> rpcProxy.refreshSuperUserGroupsConfiguration(null, + RefreshUserMappingsProtocolProtos + .RefreshSuperUserGroupsConfigurationRequestProto + .newBuilder().build()), + res -> null, Void.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java new file mode 100644 index 0000000000000..f17eab74865d0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java @@ -0,0 +1,29 @@ +/** + * 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. + */ + +/** + * This package contains the implementation of the Protocol Buffers + * protocols related to HDFS Router. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; 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..3b1d5e55781df 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 @@ -52,9 +52,9 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor { /** Time for an operation to be received in the Router. */ - private static final ThreadLocal START_TIME = new ThreadLocal<>(); + private static final ThreadLocal START_TIME = ThreadLocal.withInitial(() -> -1L); /** Time for an operation to be sent to the Namenode. */ - private static final ThreadLocal PROXY_TIME = new ThreadLocal<>(); + private static final ThreadLocal PROXY_TIME = ThreadLocal.withInitial(() -> -1L); /** Configuration for the performance monitor. */ private Configuration conf; @@ -141,6 +141,14 @@ public void startOp() { START_TIME.set(monotonicNow()); } + public static long getStartOpTime() { + return START_TIME.get(); + } + + public static void setStartOpTime(long startOpTime) { + START_TIME.set(startOpTime); + } + @Override public long proxyOp() { PROXY_TIME.set(monotonicNow()); @@ -151,6 +159,14 @@ public long proxyOp() { return Thread.currentThread().getId(); } + public static long getProxyOpTime() { + return PROXY_TIME.get(); + } + + public static void setProxyOpTime(long proxyOpTime) { + PROXY_TIME.set(proxyOpTime); + } + @Override public void proxyOpComplete(boolean success, String nsId, FederationNamenodeServiceState state) { 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 e6f41c3107a7c..f02b26796f640 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.federation.metrics; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; import static org.apache.hadoop.util.Time.now; import java.io.IOException; @@ -479,6 +480,9 @@ private String getNodesImpl(final DatanodeReportType type) { this.router.getRpcServer().getClientProtocolModule(); DatanodeStorageReport[] datanodeStorageReports = clientProtocol.getDatanodeStorageReport(type, false, dnReportTimeOut); + if (router.getRpcServer().isAsync()) { + datanodeStorageReports = syncReturn(DatanodeStorageReport[].class); + } for (DatanodeStorageReport datanodeStorageReport : datanodeStorageReports) { DatanodeInfo node = datanodeStorageReport.getDatanodeInfo(); StorageReport[] storageReports = datanodeStorageReport.getStorageReports(); @@ -512,7 +516,7 @@ private String getNodesImpl(final DatanodeReportType type) { LOG.error("Cannot get {} nodes, Router in safe mode", type); } catch (SubClusterTimeoutException e) { LOG.error("Cannot get {} nodes, subclusters timed out responding", type); - } catch (IOException e) { + } catch (Exception e) { LOG.error("Cannot get " + type + " nodes", e); } return JSON.toString(info); 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 0bc60b28fa49d..3e111b9d6f228 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 @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.federation.metrics; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; import static org.apache.hadoop.metrics2.impl.MsInfo.ProcessName; import static org.apache.hadoop.util.Time.now; @@ -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 = syncReturn(DatanodeInfo[].class); + } else { + live = rpcServer.getDatanodeReport(DatanodeReportType.LIVE, false, timeOut); + } } else { LOG.debug("Getting node usage is disabled."); } @@ -578,7 +584,7 @@ public String getNodeUsage() { StandardDeviation deviation = new StandardDeviation(); dev = deviation.evaluate(usages); } - } catch (IOException e) { + } catch (Exception e) { LOG.error("Cannot get the live nodes: {}", e.getMessage()); } 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/Quota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java index e19e51b5733ac..2c4bcc92b4719 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 @@ -139,7 +139,7 @@ public QuotaUsage getQuotaUsage(String path) throws IOException { * @return quota usage for each remote location. * @throws IOException If the quota system is disabled. */ - Map getEachQuotaUsage(String path) + protected Map getEachQuotaUsage(String path) throws IOException { rpcServer.checkOperation(OperationCategory.READ); if (!router.isQuotaEnabled()) { @@ -213,9 +213,9 @@ private boolean isMountEntry(String path) { * method will do some additional filtering. * @param path Federation path. * @return List of valid quota remote locations. - * @throws IOException + * @throws IOException If the location for this path cannot be determined. */ - private List getValidQuotaLocations(String path) + protected List getValidQuotaLocations(String path) throws IOException { final List locations = getQuotaRemoteLocations(path); @@ -252,8 +252,9 @@ private List getValidQuotaLocations(String path) * @param path Federation path of the results. * @param results Quota query result. * @return Aggregated Quota. + * @throws IOException If the quota system is disabled. */ - QuotaUsage aggregateQuota(String path, + protected QuotaUsage aggregateQuota(String path, Map results) throws IOException { long nsCount = 0; long ssCount = 0; @@ -359,7 +360,7 @@ public static boolean andByStorageType(Predicate predicate) { * federation path. * @param path Federation path. * @return List of quota remote locations. - * @throws IOException + * @throws IOException If the location for this path cannot be determined. */ private List getQuotaRemoteLocations(String path) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java index 64f27bd3ba32e..88321b92a35a8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java @@ -72,6 +72,22 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic { public static final String DFS_ROUTER_RPC_ENABLE = FEDERATION_ROUTER_PREFIX + "rpc.enable"; public static final boolean DFS_ROUTER_RPC_ENABLE_DEFAULT = true; + // HDFS Router Asynchronous RPC + public static final String DFS_ROUTER_ASYNC_RPC_ENABLE_KEY = + FEDERATION_ROUTER_PREFIX + "async.rpc.enable"; + public static final boolean DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT = false; + public static final String FEDERATION_ROUTER_ASYNC_RPC_PREFIX = + FEDERATION_ROUTER_PREFIX + "async.rpc."; + // Example: ns1:count1,ns2:count2,ns3:count3 + public static final String DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY = + FEDERATION_ROUTER_ASYNC_RPC_PREFIX + "ns.handler.count"; + public static final String DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT = ""; + public static final String DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY = + FEDERATION_ROUTER_ASYNC_RPC_PREFIX + "handler.count"; + public static final int DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT = 10; + public static final String DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY = + FEDERATION_ROUTER_ASYNC_RPC_PREFIX + "responder.count"; + public static final int DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT = 10; public static final String DFS_ROUTER_METRICS_ENABLE = FEDERATION_ROUTER_PREFIX + "metrics.enable"; 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..d9cacf2e75eba 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 @@ -21,6 +21,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_ENABLED_KEY; import static org.apache.hadoop.hdfs.server.federation.fairness.RefreshFairnessPolicyControllerHandler.HANDLER_IDENTIFIER; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; import java.io.IOException; import java.net.InetSocketAddress; @@ -627,12 +628,15 @@ private List getDestinationNameServices( Map responses = rpcClient.invokeConcurrent( locations, method, false, false, HdfsFileStatus.class); + if (rpcServer.isAsync()) { + responses = syncReturn(Map.class); + } for (RemoteLocation location : locations) { if (responses.get(location) != null) { nsIds.add(location.getNameserviceId()); } } - } catch (IOException ioe) { + } catch (Exception ioe) { LOG.error("Cannot get location for {}: {}", src, ioe.getMessage()); } 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..f2dd5dc182796 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 @@ -59,15 +59,19 @@ public RouterCacheAdmin(RouterRpcServer server) { public long addCacheDirective(CacheDirectiveInfo path, EnumSet flags) throws IOException { + Map response = invokeAddCacheDirective(path, flags); + return response.values().iterator().next(); + } + + protected Map invokeAddCacheDirective( + 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); - Map response = - rpcClient.invokeConcurrent(locations, method, false, false, long.class); - return response.values().iterator().next(); + return rpcClient.invokeConcurrent(locations, method, false, false, long.class); } public void modifyCacheDirective(CacheDirectiveInfo directive, @@ -100,6 +104,12 @@ public void removeCacheDirective(long id) throws IOException { public BatchedEntries listCacheDirectives(long prevId, CacheDirectiveInfo filter) throws IOException { + Map results = invokeListCacheDirectives(prevId, filter); + return (BatchedEntries) results.values().iterator().next(); + } + + protected Map invokeListCacheDirectives(long prevId, + CacheDirectiveInfo filter) throws IOException { rpcServer.checkOperation(NameNode.OperationCategory.READ, true); if (filter.getPath() != null) { final List locations = rpcServer @@ -107,17 +117,15 @@ public BatchedEntries listCacheDirectives(long prevId, RemoteMethod method = new RemoteMethod("listCacheDirectives", new Class[] {long.class, CacheDirectiveInfo.class}, prevId, new RemoteParam(getRemoteMap(filter, locations))); - Map response = rpcClient.invokeConcurrent( + return rpcClient.invokeConcurrent( locations, method, false, false, BatchedEntries.class); - return response.values().iterator().next(); } RemoteMethod method = new RemoteMethod("listCacheDirectives", new Class[] {long.class, CacheDirectiveInfo.class}, prevId, filter); Set nss = namenodeResolver.getNamespaces(); - Map results = rpcClient - .invokeConcurrent(nss, method, true, false, BatchedEntries.class); - return results.values().iterator().next(); + return rpcClient.invokeConcurrent( + nss, method, true, false, BatchedEntries.class); } public void addCachePool(CachePoolInfo info) throws IOException { @@ -146,13 +154,17 @@ public void removeCachePool(String cachePoolName) throws IOException { public BatchedEntries listCachePools(String prevKey) throws IOException { + Map results = invokeListCachePools(prevKey); + return results.values().iterator().next(); + } + + protected Map invokeListCachePools( + String prevKey) throws IOException { rpcServer.checkOperation(NameNode.OperationCategory.READ, true); RemoteMethod method = new RemoteMethod("listCachePools", new Class[] {String.class}, prevKey); Set nss = namenodeResolver.getNamespaces(); - Map results = rpcClient - .invokeConcurrent(nss, method, true, false, BatchedEntries.class); - return results.values().iterator().next(); + return rpcClient.invokeConcurrent(nss, method, true, false, BatchedEntries.class); } /** @@ -161,7 +173,7 @@ public BatchedEntries listCachePools(String prevKey) * @param locations the locations to map. * @return map with CacheDirectiveInfo mapped to the locations. */ - private Map getRemoteMap( + protected 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..cab4fad19096f 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 @@ -85,6 +85,10 @@ 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.router.async.AsyncErasureCoding; +import org.apache.hadoop.hdfs.server.federation.router.async.RouterAsyncCacheAdmin; +import org.apache.hadoop.hdfs.server.federation.router.async.RouterAsyncSnapshot; +import org.apache.hadoop.hdfs.server.federation.router.async.RouterAsyncStoragePolicy; import org.apache.hadoop.hdfs.server.federation.router.security.RouterSecurityManager; import org.apache.hadoop.hdfs.server.federation.store.records.MountTable; import org.apache.hadoop.hdfs.server.namenode.NameNode; @@ -166,7 +170,7 @@ public class RouterClientProtocol implements ClientProtocol { /** Router security manager to handle token operations. */ private RouterSecurityManager securityManager = null; - RouterClientProtocol(Configuration conf, RouterRpcServer rpcServer) { + public RouterClientProtocol(Configuration conf, RouterRpcServer rpcServer) { this.rpcServer = rpcServer; this.rpcClient = rpcServer.getRPCClient(); this.subclusterResolver = rpcServer.getSubclusterResolver(); @@ -194,10 +198,17 @@ public class RouterClientProtocol implements ClientProtocol { this.superGroup = conf.get( DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY, DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT); - this.erasureCoding = new ErasureCoding(rpcServer); - this.storagePolicy = new RouterStoragePolicy(rpcServer); - this.snapshotProto = new RouterSnapshot(rpcServer); - this.routerCacheAdmin = new RouterCacheAdmin(rpcServer); + if (rpcServer.isAsync()) { + this.erasureCoding = new AsyncErasureCoding(rpcServer); + this.storagePolicy = new RouterAsyncStoragePolicy(rpcServer); + this.snapshotProto = new RouterAsyncSnapshot(rpcServer); + this.routerCacheAdmin = new RouterAsyncCacheAdmin(rpcServer); + } else { + this.erasureCoding = new ErasureCoding(rpcServer); + this.storagePolicy = new RouterStoragePolicy(rpcServer); + this.snapshotProto = new RouterSnapshot(rpcServer); + this.routerCacheAdmin = new RouterCacheAdmin(rpcServer); + } this.securityManager = rpcServer.getRouterSecurityManager(); this.rbfRename = new RouterFederationRename(rpcServer, conf); this.defaultNameServiceEnabled = conf.getBoolean( @@ -347,7 +358,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( + protected List checkFaultTolerantRetry( final RemoteMethod method, final String src, final IOException ioe, final RemoteLocation excludeLoc, final List locations) throws IOException { @@ -820,7 +831,7 @@ public void renewLease(String clientName, List namespaces) /** * For {@link #getListing(String,byte[],boolean) GetLisiting} to sort results. */ - private static class GetListingComparator + protected static class GetListingComparator implements Comparator, Serializable { @Override public int compare(byte[] o1, byte[] o2) { @@ -831,6 +842,10 @@ public int compare(byte[] o1, byte[] o2) { private static GetListingComparator comparator = new GetListingComparator(); + public static GetListingComparator getComparator() { + return comparator; + } + @Override public DirectoryListing getListing(String src, byte[] startAfter, boolean needLocation) throws IOException { @@ -1104,7 +1119,7 @@ public DatanodeStorageReport[] getDatanodeStorageReport( return mergeDtanodeStorageReport(dnSubcluster); } - private DatanodeStorageReport[] mergeDtanodeStorageReport( + protected DatanodeStorageReport[] mergeDtanodeStorageReport( Map dnSubcluster) { // Avoid repeating machines in multiple subclusters Map datanodesMap = new LinkedHashMap<>(); @@ -1335,20 +1350,23 @@ Map> getAllLocations(String path) throws IOExceptio } /** - * Get all the locations of the path for {@link this#getContentSummary(String)}. + * Get all the locations of the path for {@link RouterClientProtocol#getContentSummary(String)}. * For example, there are some mount points: - * /a -> ns0 -> /a - * /a/b -> ns0 -> /a/b - * /a/b/c -> ns1 -> /a/b/c + *

+ * /a - [ns0 - /a] + * /a/b - [ns0 - /a/b] + * /a/b/c - [ns1 - /a/b/c] + *

* When the path is '/a', the result of locations should be * [RemoteLocation('/a', ns0, '/a'), RemoteLocation('/a/b/c', ns1, '/a/b/c')] * When the path is '/b', will throw NoLocationException. + * * @param path the path to get content summary * @return one list contains all the remote location - * @throws IOException + * @throws IOException if an I/O error occurs */ @VisibleForTesting - List getLocationsForContentSummary(String path) throws IOException { + protected List getLocationsForContentSummary(String path) throws IOException { // Try to get all the locations of the path. final Map> ns2Locations = getAllLocations(path); if (ns2Locations.isEmpty()) { @@ -2039,7 +2057,7 @@ public HAServiceProtocol.HAServiceState getHAServiceState() { * replacement value. * @throws IOException If the dst paths could not be determined. */ - private RemoteParam getRenameDestinations( + protected RemoteParam getRenameDestinations( final List srcLocations, final List dstLocations) throws IOException { @@ -2087,7 +2105,7 @@ private RemoteLocation getFirstMatchingLocation(RemoteLocation location, * @param summaries Collection of individual summaries. * @return Aggregated content summary. */ - private ContentSummary aggregateContentSummary( + protected ContentSummary aggregateContentSummary( Collection summaries) { if (summaries.size() == 1) { return summaries.iterator().next(); @@ -2142,7 +2160,7 @@ private ContentSummary aggregateContentSummary( * everywhere. * @throws IOException If all the locations throw an exception. */ - private HdfsFileStatus getFileInfoAll(final List locations, + protected HdfsFileStatus getFileInfoAll(final List locations, final RemoteMethod method) throws IOException { return getFileInfoAll(locations, method, -1); } @@ -2157,7 +2175,7 @@ private HdfsFileStatus getFileInfoAll(final List locations, * everywhere. * @throws IOException If all the locations throw an exception. */ - private HdfsFileStatus getFileInfoAll(final List locations, + protected HdfsFileStatus getFileInfoAll(final List locations, final RemoteMethod method, long timeOutMs) throws IOException { // Get the file info from everybody @@ -2186,12 +2204,11 @@ private HdfsFileStatus getFileInfoAll(final List locations, /** * Get the permissions for the parent of a child with given permissions. - * Add implicit u+wx permission for parent. This is based on - * @{FSDirMkdirOp#addImplicitUwx}. + * Add implicit u+wx permission for parent. This is based on FSDirMkdirOp#addImplicitUwx. * @param mask The permission mask of the child. * @return The permission mask of the parent. */ - private static FsPermission getParentPermission(final FsPermission mask) { + protected static FsPermission getParentPermission(final FsPermission mask) { FsPermission ret = new FsPermission( mask.getUserAction().or(FsAction.WRITE_EXECUTE), mask.getGroupAction(), @@ -2208,7 +2225,7 @@ private static FsPermission getParentPermission(final FsPermission mask) { * @return New HDFS file status representing a mount point. */ @VisibleForTesting - HdfsFileStatus getMountPointStatus( + protected HdfsFileStatus getMountPointStatus( String name, int childrenNum, long date) { return getMountPointStatus(name, childrenNum, date, true); } @@ -2223,7 +2240,7 @@ HdfsFileStatus getMountPointStatus( * @return New HDFS file status representing a mount point. */ @VisibleForTesting - HdfsFileStatus getMountPointStatus( + protected HdfsFileStatus getMountPointStatus( String name, int childrenNum, long date, boolean setPath) { long modTime = date; long accessTime = date; @@ -2300,7 +2317,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) { + protected Map getMountPointDates(String path) { Map ret = new TreeMap<>(); if (subclusterResolver instanceof MountTableResolver) { try { @@ -2361,9 +2378,15 @@ private long getModifiedTime(Map ret, String path, } /** - * Get listing on remote locations. + * Get a partial listing of the indicated directory. + * + * @param src the directory name + * @param startAfter the name to start after + * @param needLocation if blockLocations need to be returned + * @return a partial listing starting after startAfter + * @throws IOException if other I/O error occurred */ - private List> getListingInt( + protected List> getListingInt( String src, byte[] startAfter, boolean needLocation) throws IOException { try { List locations = @@ -2400,9 +2423,9 @@ private List> getListingInt( * @param startAfter starting listing from client, used to define listing * start boundary * @param remainingEntries how many entries left from subcluster - * @return + * @return true if should add mount point, otherwise false; */ - private static boolean shouldAddMountPoint( + protected static boolean shouldAddMountPoint( byte[] mountPoint, byte[] lastEntry, byte[] startAfter, int remainingEntries) { if (comparator.compare(mountPoint, startAfter) > 0 && @@ -2425,7 +2448,7 @@ private static boolean shouldAddMountPoint( * @throws IOException if unable to get the file status. */ @VisibleForTesting - boolean isMultiDestDirectory(String src) throws IOException { + protected boolean isMultiDestDirectory(String src) throws IOException { try { if (rpcServer.isPathAll(src)) { List locations; @@ -2449,4 +2472,56 @@ boolean isMultiDestDirectory(String src) throws IOException { public int getRouterFederationRenameCount() { return rbfRename.getRouterFederationRenameCount(); } + + public RouterRpcServer getRpcServer() { + return rpcServer; + } + + public RouterRpcClient getRpcClient() { + return rpcClient; + } + + public FileSubclusterResolver getSubclusterResolver() { + return subclusterResolver; + } + + public ActiveNamenodeResolver getNamenodeResolver() { + return namenodeResolver; + } + + public long getServerDefaultsLastUpdate() { + return serverDefaultsLastUpdate; + } + + public long getServerDefaultsValidityPeriod() { + return serverDefaultsValidityPeriod; + } + + public boolean isAllowPartialList() { + return allowPartialList; + } + + public long getMountStatusTimeOut() { + return mountStatusTimeOut; + } + + public String getSuperUser() { + return superUser; + } + + public String getSuperGroup() { + return superGroup; + } + + public RouterStoragePolicy getStoragePolicy() { + return storagePolicy; + } + + public void setServerDefaultsLastUpdate(long serverDefaultsLastUpdate) { + this.serverDefaultsLastUpdate = serverDefaultsLastUpdate; + } + + public RouterFederationRename getRbfRename() { + return rbfRename; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterFederationRename.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterFederationRename.java index aafb685b88626..772e725788826 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterFederationRename.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterFederationRename.java @@ -93,7 +93,7 @@ public RouterFederationRename(RouterRpcServer rpcServer, Configuration conf) { * @throws IOException if rename fails. * @return true if rename succeeds. */ - boolean routerFedRename(final String src, final String dst, + public boolean routerFedRename(final String src, final String dst, final List srcLocations, final List dstLocations) throws IOException { if (!rpcServer.isEnableRenameAcrossNamespace()) { 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..124d55586d86b 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 @@ -41,6 +41,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; + /** * Service to periodically update the {@link RouterQuotaUsage} * cached information in the {@link Router}. @@ -99,6 +101,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 = syncReturn(HdfsFileStatus.class); + } if (ret == null || ret.getModificationTime() == 0) { long[] zeroConsume = new long[StorageType.values().length]; currentQuotaUsage = @@ -113,6 +118,9 @@ protected void periodicInvoke() { Quota quotaModule = this.rpcServer.getQuotaModule(); Map usageMap = quotaModule.getEachQuotaUsage(src); + if (this.rpcServer.isAsync()) { + usageMap = (Map)syncReturn(Map.class); + } currentQuotaUsage = quotaModule.aggregateQuota(src, usageMap); remoteQuotaUsage.putAll(usageMap); } catch (IOException ioe) { @@ -136,6 +144,8 @@ protected void periodicInvoke() { } } catch (IOException e) { LOG.error("Quota cache updated error.", e); + } catch (Exception e) { + LOG.error(e.toString()); } } 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..c7c3699f33ec7 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 @@ -400,7 +400,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 +462,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 { @@ -526,11 +526,12 @@ public Object invokeMethod( if (rpcMonitor != null) { rpcMonitor.proxyOp(); } - boolean failover = false; - boolean shouldUseObserver = useObserver; + + ExecutionStatus status = new ExecutionStatus(false, useObserver); Map ioes = new LinkedHashMap<>(); for (FederationNamenodeContext namenode : namenodes) { - if (!shouldUseObserver && (namenode.getState() == FederationNamenodeServiceState.OBSERVER)) { + if (!status.isShouldUseObserver() + && (namenode.getState() == FederationNamenodeServiceState.OBSERVER)) { continue; } ConnectionContext connection = null; @@ -541,83 +542,12 @@ public Object invokeMethod( ProxyAndInfo client = connection.getClient(); final Object proxy = client.getProxy(); - ret = invoke(nsId, namenode, useObserver, 0, method, proxy, params); - if (failover && - FederationNamenodeServiceState.OBSERVER != namenode.getState()) { - // Success on alternate server, update - InetSocketAddress address = client.getAddress(); - namenodeResolver.updateActiveNamenode(nsId, address); - } - if (this.rpcMonitor != null) { - this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState()); - } - if (this.router.getRouterClientMetrics() != null) { - this.router.getRouterClientMetrics().incInvokedMethod(method); - } + ret = invoke(namenode, useObserver, 0, method, proxy, params); + postProcessResult(method, status, namenode, nsId, client); return ret; } catch (IOException ioe) { ioes.put(namenode, ioe); - if (ioe instanceof ObserverRetryOnActiveException) { - LOG.info("Encountered ObserverRetryOnActiveException from {}." - + " Retry active namenode directly.", namenode); - shouldUseObserver = false; - } else if (ioe instanceof StandbyException) { - // Fail over indicated by retry policy and/or NN - if (this.rpcMonitor != null) { - this.rpcMonitor.proxyOpFailureStandby(nsId); - } - failover = true; - } else if (isUnavailableException(ioe)) { - if (this.rpcMonitor != null) { - this.rpcMonitor.proxyOpFailureCommunicate(nsId); - } - if (FederationNamenodeServiceState.OBSERVER == namenode.getState()) { - namenodeResolver.updateUnavailableNamenode(nsId, - NetUtils.createSocketAddr(namenode.getRpcAddress())); - } else { - failover = true; - } - } else if (ioe instanceof RemoteException) { - if (this.rpcMonitor != null) { - this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState()); - } - RemoteException re = (RemoteException) ioe; - ioe = re.unwrapRemoteException(); - ioe = getCleanException(ioe); - // RemoteException returned by NN - throw ioe; - } else if (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 se; - } else if (ioe instanceof NoNamenodesAvailableException) { - IOException cause = (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(cause)) { - this.namenodeResolver.rotateCache(nsId, namenode, useObserver); - } - // Throw RetriableException so that client can retry - throw 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()); - } - throw ioe; - } + handleInvokeMethodIOException(namenode, ioe, status, useObserver); } finally { if (connection != null) { connection.release(); @@ -628,6 +558,24 @@ public Object invokeMethod( this.rpcMonitor.proxyOpComplete(false, null, null); } + return handlerAllNamenodeFail(namenodes, method, ioes, params); + } + + /** + * All namenodes cannot successfully process the RPC request, + * throw corresponding exceptions according to the exception type of each namenode. + * + * @param namenodes A prioritized list of namenodes within the same nameservice. + * @param method Remote ClientProtocol method to invoke. + * @param ioes The exception type of each namenode. + * @param params Variable list of parameters matching the method. + * @return null + * @throws IOException Corresponding IOException according to the + * exception type of each namenode. + */ + protected Object handlerAllNamenodeFail( + List namenodes, Method method, + Map ioes, Object[] params) throws IOException { // All namenodes were unavailable or in standby String msg = "No namenode available to invoke " + method.getName() + " " + Arrays.deepToString(params) + " in " + namenodes + " from " + @@ -658,14 +606,120 @@ public Object invokeMethod( } } + /** + * The RPC request is successfully processed by the NameNode, the NameNode status + * in the router cache is updated according to the ExecutionStatus. + * + * @param method Remote method to invoke. + * @param status Current execution status. + * @param namenode The namenode that successfully processed this RPC request. + * @param nsId Nameservice ID. + * @param client Connection client. + * @throws IOException If the state store cannot be accessed. + */ + protected void postProcessResult(Method method, ExecutionStatus status, + FederationNamenodeContext namenode, String nsId, ProxyAndInfo client) throws IOException { + if (status.isFailOver() && + FederationNamenodeServiceState.OBSERVER != namenode.getState()) { + // Success on alternate server, update + InetSocketAddress address = client.getAddress(); + namenodeResolver.updateActiveNamenode(nsId, address); + } + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState()); + } + if (this.router.getRouterClientMetrics() != null) { + this.router.getRouterClientMetrics().incInvokedMethod(method); + } + } + + /** + * The RPC request to the NameNode throws an exception, + * handle it according to the type of exception. + * + * @param namenode The namenode that processed this RPC request. + * @param ioe The exception thrown by this RPC request. + * @param status The current execution status. + * @param useObserver Whether to use observer namenodes. + * @throws IOException If it cannot invoke the method. + */ + protected void handleInvokeMethodIOException(final FederationNamenodeContext namenode, + IOException ioe, final ExecutionStatus status, boolean useObserver) throws IOException { + String nsId = namenode.getNameserviceId(); + String rpcAddress = namenode.getRpcAddress(); + if (ioe instanceof ObserverRetryOnActiveException) { + LOG.info("Encountered ObserverRetryOnActiveException from {}." + + " Retry active namenode directly.", namenode); + status.setShouldUseObserver(false); + } else if (ioe instanceof StandbyException) { + // Fail over indicated by retry policy and/or NN + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpFailureStandby(nsId); + } + status.setFailOver(true); + } else if (isUnavailableException(ioe)) { + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpFailureCommunicate(nsId); + } + if (FederationNamenodeServiceState.OBSERVER == namenode.getState()) { + namenodeResolver.updateUnavailableNamenode(nsId, + NetUtils.createSocketAddr(namenode.getRpcAddress())); + } else { + status.setFailOver(true); + } + } else if (ioe instanceof RemoteException) { + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpComplete(true, nsId, namenode.getState()); + } + RemoteException re = (RemoteException) ioe; + ioe = re.unwrapRemoteException(); + ioe = getCleanException(ioe); + // RemoteException returned by NN + throw ioe; + } else if (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 se; + } else if (ioe instanceof NoNamenodesAvailableException) { + IOException cause = (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(cause)) { + this.namenodeResolver.rotateCache(nsId, namenode, useObserver); + } + // Throw RetriableException so that client can retry + throw 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()); + } + throw ioe; + } + } + /** * For tracking some information about the actual client. * It adds trace info "clientIp:ip", "clientPort:port", * "clientId:id", "clientCallId:callId" and "realUser:userName" * in the caller context, removing the old values if they were * already present. + * + * @param ugi User group information. */ - 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(); @@ -706,7 +760,8 @@ private void addClientInfoToCallerContext(UserGroupInformation ugi) { * Re-throws exceptions generated by the remote RPC call as either * RemoteException or IOException. * - * @param nsId Identifier for the namespace + * @param namenode namenode context. + * @param listObserverFirst Observer read case, observer NN will be ranked first. * @param retryCount Current retry times * @param method Method to invoke * @param obj Target object for the method @@ -714,8 +769,8 @@ private void addClientInfoToCallerContext(UserGroupInformation ugi) { * @return Response from the remote server * @throws IOException If error occurs. */ - private Object invoke( - String nsId, FederationNamenodeContext namenode, Boolean listObserverFirst, + protected Object invoke( + FederationNamenodeContext namenode, Boolean listObserverFirst, int retryCount, final Method method, final Object obj, final Object... params) throws IOException { try { @@ -725,36 +780,58 @@ private Object invoke( return null; } catch (InvocationTargetException e) { Throwable cause = e.getCause(); - if (cause instanceof IOException) { - IOException ioe = (IOException) cause; - - // Check if we should retry. - RetryDecision decision = shouldRetry(ioe, retryCount, nsId, namenode, listObserverFirst); - if (decision == RetryDecision.RETRY) { - if (this.rpcMonitor != null) { - this.rpcMonitor.proxyOpRetries(); - } + return handlerInvokeException(namenode, listObserverFirst, + retryCount, method, obj, cause, params); + } + } - // retry - return invoke(nsId, namenode, listObserverFirst, ++retryCount, method, obj, params); - } else if (decision == RetryDecision.FAILOVER_AND_RETRY) { - // failover, invoker looks for standby exceptions for failover. - if (ioe instanceof StandbyException) { - throw ioe; - } else if (isUnavailableException(ioe)) { - throw ioe; - } else { - throw new StandbyException(ioe.getMessage()); - } - } else { + /** + * Handle the exception when an RPC request to the NameNode throws an exception. + * + * @param namenode namenode context. + * @param listObserverFirst Observer read case, observer NN will be ranked first. + * @param retryCount Current retry times + * @param method Method to invoke + * @param obj Target object for the method + * @param e The exception thrown by the current invocation. + * @param params Variable parameters + * @return Response from the remote server + * @throws IOException If error occurs. + */ + protected Object handlerInvokeException(FederationNamenodeContext namenode, + Boolean listObserverFirst, int retryCount, Method method, Object obj, + Throwable e, Object[] params) throws IOException { + String nsId = namenode.getNameserviceId(); + if (e instanceof IOException) { + IOException ioe = (IOException) e; + + // Check if we should retry. + RetryDecision decision = shouldRetry(ioe, retryCount, nsId, namenode, listObserverFirst); + if (decision == RetryDecision.RETRY) { + if (this.rpcMonitor != null) { + this.rpcMonitor.proxyOpRetries(); + } + + // retry + return invoke(namenode, listObserverFirst, ++retryCount, method, obj, params); + } else if (decision == RetryDecision.FAILOVER_AND_RETRY) { + // failover, invoker looks for standby exceptions for failover. + if (ioe instanceof StandbyException) { throw ioe; + } else if (isUnavailableException(ioe)) { + throw ioe; + } else { + throw new StandbyException(ioe.getMessage()); } } else { - throw new IOException(e); + throw ioe; } + } else { + throw new IOException(e); } } + /** * Check if the exception comes from an unavailable subcluster. * @param ioe IOException to check. @@ -817,7 +894,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 +1262,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 +1328,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 +1346,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) { @@ -1414,7 +1491,26 @@ public Map invokeConcurrent( throws IOException { final List> results = invokeConcurrent( locations, method, standby, timeOutMs, clazz); + return postProcessResult(requireResponse, results); + } + /** + * Post-process the results returned by + * {@link RouterRpcClient#invokeConcurrent(Collection, RemoteMethod, boolean, long, Class)}. + * + * @param requireResponse If true an exception will be thrown if all calls do + * not complete. If false exceptions are ignored and all data results + * successfully received are returned. + * @param results Result of invoking the method per subcluster (list of results), + * This includes the exception for each remote location. + * @return Result of invoking the method per subcluster: nsId to result. + * @param The type of the remote location. + * @param The type of the remote method return. + * @throws IOException If requiredResponse=true and any of the calls throw an + * exception. + */ + protected static Map postProcessResult( + boolean requireResponse, List> results) throws IOException { // Go over the results and exceptions final Map ret = new TreeMap<>(); final List thrownExceptions = new ArrayList<>(); @@ -1480,27 +1576,10 @@ public Map invokeConcurrent( 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); - R result = (R) invokeMethod( - ugi, namenodes, isObserverRead, proto, m, paramList); - RemoteResult remoteResult = new RemoteResult<>(location, result); - return Collections.singletonList(remoteResult); - } catch (IOException ioe) { - // Localize the exception - throw processException(ioe, location); - } finally { - releasePermit(ns, ugi, method, controller); - } + return invokeSingle(locations.iterator().next(), method); } + RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController(); + acquirePermit(CONCURRENT_NS, ugi, method, controller); List orderedLocations = new ArrayList<>(); List> callables = new ArrayList<>(); @@ -1551,8 +1630,29 @@ public Map invokeConcurrent( this.router.getRouterClientMetrics().incInvokedConcurrent(m); } - RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController(); - acquirePermit(CONCURRENT_NS, ugi, method, controller); + return getRemoteResults(method, timeOutMs, controller, orderedLocations, callables); + } + + /** + * Invokes multiple concurrent proxy calls to different clients. Returns an + * array of results. + * + * @param The type of the remote location. + * @param The type of the remote method return. + * @param method The remote method and parameters to invoke. + * @param timeOutMs Timeout for each individual call. + * @param controller Fairness manager to control handlers assigned per NS. + * @param orderedLocations List of remote locations to call concurrently. + * @param callables Invoke method for each NameNode. + * @return Result of invoking the method per subcluster (list of results), + * This includes the exception for each remote location. + * @throws IOException If there are errors invoking the method. + */ + protected List> getRemoteResults( + RemoteMethod method, long timeOutMs, RouterRpcFairnessPolicyController controller, + List orderedLocations, List> callables) throws IOException { + final UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + final Method m = method.getMethod(); try { List> futures = null; if (timeOutMs > 0) { @@ -1561,42 +1661,7 @@ public Map invokeConcurrent( } else { futures = executorService.invokeAll(callables); } - List> results = new ArrayList<>(); - for (int i=0; i future = futures.get(i); - R result = (R) future.get(); - results.add(new RemoteResult<>(location, result)); - } catch (CancellationException ce) { - T loc = orderedLocations.get(i); - String msg = "Invocation to \"" + loc + "\" for \"" - + method.getMethodName() + "\" timed out"; - LOG.error(msg); - IOException ioe = new SubClusterTimeoutException(msg); - results.add(new RemoteResult<>(location, ioe)); - } catch (ExecutionException ex) { - Throwable cause = ex.getCause(); - LOG.debug("Cannot execute {} in {}: {}", - m.getName(), location, cause.getMessage()); - - // Convert into IOException if needed - IOException ioe = null; - if (cause instanceof IOException) { - 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; + return processFutures(method, m, orderedLocations, futures); } catch (RejectedExecutionException e) { if (rpcMonitor != null) { rpcMonitor.proxyOpFailureClientOverloaded(); @@ -1616,6 +1681,99 @@ public Map invokeConcurrent( } } + /** + * Handle all futures during the invokeConcurrent call process. + * + * @param The type of the remote location. + * @param The type of the remote method return. + * @param method The remote method and parameters to invoke. + * @param m The method to invoke. + * @param orderedLocations List of remote locations to call concurrently. + * @param futures all futures during the invokeConcurrent call process. + * @return Result of invoking the method per subcluster (list of results), + * This includes the exception for each remote location. + * @throws InterruptedException if the current thread was interrupted while waiting. + */ + protected List> processFutures( + RemoteMethod method, Method m, final List orderedLocations, + final List> futures) throws InterruptedException{ + List> results = new ArrayList<>(); + for (int i = 0; i< futures.size(); i++) { + T location = orderedLocations.get(i); + try { + Future future = futures.get(i); + R result = (R) future.get(); + results.add(new RemoteResult<>(location, result)); + } catch (CancellationException ce) { + T loc = orderedLocations.get(i); + String msg = "Invocation to \"" + loc + "\" for \"" + + method.getMethodName() + "\" timed out"; + LOG.error(msg); + IOException ioe = new SubClusterTimeoutException(msg); + results.add(new RemoteResult<>(location, ioe)); + } catch (ExecutionException ex) { + Throwable cause = ex.getCause(); + LOG.debug("Cannot execute {} in {}: {}", + m.getName(), location, cause.getMessage()); + + // Convert into IOException if needed + IOException ioe = null; + if (cause instanceof IOException) { + 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; + } + + /** + * Invokes a ClientProtocol method against the specified namespace. + *

+ * Re-throws exceptions generated by the remote RPC call as either + * RemoteException or IOException. + * + * @param The type of the remote location. + * @param The type of the remote method return. + * @param location RemoteLocation to invoke. + * @param method The remote method and parameters to invoke. + * @return Result of invoking the method per subcluster (list of results), + * This includes the exception for each remote location. + * @throws IOException If there are errors invoking the method. + */ + public List> invokeSingle( + T location, RemoteMethod method) throws IOException { + final UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + final Method m = method.getMethod(); + 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); + R result = (R) invokeMethod( + ugi, namenodes, isObserverRead, proto, m, paramList); + RemoteResult remoteResult = new RemoteResult<>(location, result); + return Collections.singletonList(remoteResult); + } catch (IOException ioe) { + // Localize the exception + throw processException(ioe, location); + } finally { + releasePermit(ns, ugi, method, controller); + } + } + /** * Transfer origin thread local context which is necessary to current * worker thread when invoking method concurrently by executor service. @@ -1624,7 +1782,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 +1833,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 +1866,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 +1940,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, + public List getOrderedNamenodes(String nsId, boolean isObserverRead) throws IOException { final List namenodes; @@ -1802,7 +1960,7 @@ && isNamespaceStateIdFresh(nsId) return namenodes; } - private boolean isObserverReadEligible(String nsId, Method method) { + protected boolean isObserverReadEligible(String nsId, Method method) { return isReadCall(method) && isNamespaceObserverReadEligible(nsId); } @@ -1811,7 +1969,7 @@ private boolean isObserverReadEligible(String nsId, Method method) { * @param nsId namespaceID * @return whether the 'namespace' has observer reads enabled. */ - boolean isNamespaceObserverReadEligible(String nsId) { + public boolean isNamespaceObserverReadEligible(String nsId) { return observerReadEnabledDefault != observerReadEnabledOverrides.contains(nsId); } @@ -1857,7 +2015,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; } @@ -1868,4 +2026,61 @@ private boolean shouldRotateCache(IOException ioe) { } return isUnavailableException(ioe); } + + + /** + * The {@link ExecutionStatus} class is a utility class used to track the status of + * execution operations performed by the {@link RouterRpcClient}. + * It encapsulates the state of an operation, including whether it has completed, + * if a failover to a different NameNode should be attempted, and if an observer + * NameNode should be used for the operation. + * + *

The status is represented by a flag that indicate the current state of + * the execution. The flag can be checked individually to determine how to + * proceed with the operation or to handle its results. + */ + protected static class ExecutionStatus { + + /** A byte field used to store the state flags. */ + private byte flag; + private static final byte FAIL_OVER_BIT = 1; + private static final byte SHOULD_USE_OBSERVER_BIT = 2; + private static final byte COMPLETE_BIT = 4; + + public ExecutionStatus() { + this(false, false); + } + + public ExecutionStatus(boolean failOver, boolean shouldUseObserver) { + this.flag = 0; + setFailOver(failOver); + setShouldUseObserver(shouldUseObserver); + setComplete(false); + } + + public void setFailOver(boolean failOver) { + flag = (byte) (failOver ? (flag | FAIL_OVER_BIT) : (flag & ~FAIL_OVER_BIT)); + } + + public void setShouldUseObserver(boolean shouldUseObserver) { + flag = (byte) (shouldUseObserver ? + (flag | SHOULD_USE_OBSERVER_BIT) : (flag & ~SHOULD_USE_OBSERVER_BIT)); + } + + public void setComplete(boolean complete) { + flag = (byte) (complete ? (flag | COMPLETE_BIT) : (flag & ~COMPLETE_BIT)); + } + + public boolean isFailOver() { + return (flag & FAIL_OVER_BIT) != 0; + } + + public boolean isShouldUseObserver() { + return (flag & SHOULD_USE_OBSERVER_BIT) != 0; + } + + public boolean isComplete() { + return (flag & COMPLETE_BIT) != 0; + } + } } 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 0b9d5eaa6897a..77bebab4ade71 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 @@ -18,6 +18,16 @@ package org.apache.hadoop.hdfs.server.federation.router; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY; +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.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT; @@ -28,9 +38,14 @@ import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_READER_QUEUE_SIZE_KEY; import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DN_REPORT_CACHE_EXPIRE; 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.RouterFederationRename.RouterRenameOption; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCatch; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncForEach; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncTry; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI; import java.io.FileNotFoundException; @@ -43,22 +58,40 @@ import java.util.ArrayList; import java.util.Collection; import java.util.EnumSet; +import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +import org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil; +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.server.federation.router.async.AsyncQuota; +import org.apache.hadoop.hdfs.server.federation.router.async.RouterAsyncClientProtocol; +import org.apache.hadoop.hdfs.server.federation.router.async.RouterAsyncNamenodeProtocol; +import org.apache.hadoop.hdfs.server.federation.router.async.RouterAsyncRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.async.RouterAsyncUserProtocol; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncCatchFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.CatchFunction; 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; @@ -180,6 +213,7 @@ import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB; import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB; import org.apache.hadoop.util.ReflectionUtils; +import org.checkerframework.checker.nullness.qual.NonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -200,6 +234,8 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, private static final Logger LOG = LoggerFactory.getLogger(RouterRpcServer.class); + /** Name service keyword to identify fan-out calls. */ + public static final String CONCURRENT_NS = "concurrent"; /** Configuration for the RPC server. */ private Configuration conf; @@ -256,6 +292,12 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, private RouterRenameOption routerRenameOption; /** Schedule the router federation rename jobs. */ private BalanceProcedureScheduler fedRenameScheduler; + private boolean enableAsync; + private Map nsAsyncHandlerCount = new ConcurrentHashMap<>(); + private Map asyncRouterHandlerExecutors = new ConcurrentHashMap<>(); + private ExecutorService routerAsyncResponderExecutor; + private ExecutorService routerDefaultAsyncHandlerExecutor; + /** * Construct a router RPC server. * @@ -265,6 +307,7 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol, * @param fileResolver File resolver to resolve file paths to subclusters. * @throws IOException If the RPC server could not be created. */ + @SuppressWarnings("checkstyle:MethodLength") public RouterRpcServer(Configuration conf, Router router, ActiveNamenodeResolver nnResolver, FileSubclusterResolver fileResolver) throws IOException { @@ -285,6 +328,12 @@ public RouterRpcServer(Configuration conf, Router router, int handlerQueueSize = this.conf.getInt(DFS_ROUTER_HANDLER_QUEUE_SIZE_KEY, DFS_ROUTER_HANDLER_QUEUE_SIZE_DEFAULT); + this.enableAsync = conf.getBoolean(DFS_ROUTER_ASYNC_RPC_ENABLE_KEY, + DFS_ROUTER_ASYNC_RPC_ENABLE_DEFAULT); + LOG.info("Router enable async rpc: {}", this.enableAsync); + if (this.enableAsync) { + initAsyncThreadPools(conf); + } // Override Hadoop Common IPC setting int readerQueueSize = this.conf.getInt(DFS_ROUTER_READER_QUEUE_SIZE_KEY, DFS_ROUTER_READER_QUEUE_SIZE_DEFAULT); @@ -294,26 +343,39 @@ public RouterRpcServer(Configuration conf, Router router, RPC.setProtocolEngine(this.conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine2.class); - ClientNamenodeProtocolServerSideTranslatorPB - clientProtocolServerTranslator = - new ClientNamenodeProtocolServerSideTranslatorPB(this); + clientProtocolServerTranslator = null; + NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = null; + RefreshUserMappingsProtocolServerSideTranslatorPB refreshUserMappingXlator = null; + GetUserMappingsProtocolServerSideTranslatorPB getUserMappingXlator = null; + if (isAsync()) { + clientProtocolServerTranslator = + new RouterClientNamenodeProtocolServerSideTranslatorPB(this); + namenodeProtocolXlator = + new RouterNamenodeProtocolServerSideTranslatorPB(this); + refreshUserMappingXlator = + new RouterRefreshUserMappingsProtocolServerSideTranslatorPB(this); + getUserMappingXlator = + new RouterGetUserMappingsProtocolServerSideTranslatorPB(this); + } else { + clientProtocolServerTranslator = new ClientNamenodeProtocolServerSideTranslatorPB(this); + namenodeProtocolXlator = + new NamenodeProtocolServerSideTranslatorPB(this); + refreshUserMappingXlator = + new RefreshUserMappingsProtocolServerSideTranslatorPB(this); + getUserMappingXlator = + new GetUserMappingsProtocolServerSideTranslatorPB(this); + } BlockingService clientNNPbService = ClientNamenodeProtocol .newReflectiveBlockingService(clientProtocolServerTranslator); - NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = - new NamenodeProtocolServerSideTranslatorPB(this); BlockingService nnPbService = NamenodeProtocolService .newReflectiveBlockingService(namenodeProtocolXlator); - RefreshUserMappingsProtocolServerSideTranslatorPB refreshUserMappingXlator = - new RefreshUserMappingsProtocolServerSideTranslatorPB(this); BlockingService refreshUserMappingService = RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService. newReflectiveBlockingService(refreshUserMappingXlator); - GetUserMappingsProtocolServerSideTranslatorPB getUserMappingXlator = - new GetUserMappingsProtocolServerSideTranslatorPB(this); BlockingService getUserMappingService = GetUserMappingsProtocolProtos.GetUserMappingsProtocolService. newReflectiveBlockingService(getUserMappingXlator); @@ -369,8 +431,7 @@ public RouterRpcServer(Configuration conf, Router router, NotReplicatedYetException.class, IOException.class, ConnectException.class, - RetriableException.class, - PathIsNotEmptyDirectoryException.class); + RetriableException.class); this.rpcServer.addSuppressedLoggingExceptions( StandbyException.class, UnresolvedPathException.class); @@ -393,14 +454,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 (this.enableAsync) { + this.rpcClient = new RouterAsyncRpcClient(this.conf, this.router, + this.namenodeResolver, this.rpcMonitor, routerStateIdContext); + this.clientProto = new RouterAsyncClientProtocol(conf, this); + this.nnProto = new RouterAsyncNamenodeProtocol(this); + this.routerProto = new RouterAsyncUserProtocol(this); + this.quotaCall = new AsyncQuota(this.router, this); + } else { + this.rpcClient = new RouterRpcClient(this.conf, this.router, + this.namenodeResolver, this.rpcMonitor, routerStateIdContext); + this.clientProto = new RouterClientProtocol(conf, this); + this.nnProto = new RouterNamenodeProtocol(this); + this.routerProto = new RouterUserProtocol(this); + this.quotaCall = new Quota(this.router, this); + } long dnCacheExpire = conf.getTimeDuration( DN_REPORT_CACHE_EXPIRE, @@ -430,6 +498,81 @@ public RouterRpcServer(Configuration conf, Router router, initRouterFedRename(); } + /** + * Init router async handlers and router async responders. + * @param configuration the configuration. + */ + public void initAsyncThreadPools(Configuration configuration) { + LOG.info("Begin initialize asynchronous handler and responder thread pool."); + initNsAsyncHandlerCount(); + Set allConfiguredNS = FederationUtil.getAllConfiguredNS(configuration); + Set unassignedNS = new HashSet<>(); + allConfiguredNS.add(CONCURRENT_NS); + + for (String nsId : allConfiguredNS) { + int dedicatedHandlers = nsAsyncHandlerCount.getOrDefault(nsId, 0); + LOG.info("Dedicated handlers {} for ns {} ", dedicatedHandlers, nsId); + if (dedicatedHandlers > 0) { + initAsyncHandlerThreadPools4Ns(nsId, dedicatedHandlers); + LOG.info("Assigned {} async handlers to nsId {} ", dedicatedHandlers, nsId); + } else { + unassignedNS.add(nsId); + } + } + + int asyncHandlerCountDefault = configuration.getInt(DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY, + DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_DEFAULT); + + if (!unassignedNS.isEmpty()) { + LOG.warn("Async handler unassigned ns: {}", unassignedNS); + LOG.info("Use default async handler count {} for unassigned ns.", asyncHandlerCountDefault); + for (String nsId : unassignedNS) { + initAsyncHandlerThreadPools4Ns(nsId, asyncHandlerCountDefault); + } + } + + int asyncResponderCount = configuration.getInt(DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY, + DFS_ROUTER_ASYNCRPC_RESPONDER_COUNT_DEFAULT); + if (routerAsyncResponderExecutor == null) { + LOG.info("Initialize router async responder count: {}", asyncResponderCount); + routerAsyncResponderExecutor = Executors.newFixedThreadPool( + asyncResponderCount, new AsyncThreadFactory("Router Async Responder #")); + } + AsyncRpcProtocolPBUtil.setAsyncResponderExecutor(routerAsyncResponderExecutor); + + if (routerDefaultAsyncHandlerExecutor == null) { + LOG.info("init router async default executor handler count: {}", asyncHandlerCountDefault); + routerDefaultAsyncHandlerExecutor = Executors.newFixedThreadPool( + asyncHandlerCountDefault, new AsyncThreadFactory("Router Async Default Handler #")); + } + } + + private void initNsAsyncHandlerCount() { + String configNsHandler = conf.get(DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY, + DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_DEFAULT); + if (StringUtils.isEmpty(configNsHandler)) { + LOG.error( + "The value of config key: {} is empty. Will use default conf.", + DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY); + } + String[] nsHandlers = configNsHandler.split(","); + for (String nsHandlerInfo : nsHandlers) { + String[] nsHandlerItems = nsHandlerInfo.split(":"); + if (nsHandlerItems.length != 2 || StringUtils.isBlank(nsHandlerItems[0]) || + !StringUtils.isNumeric(nsHandlerItems[1])) { + LOG.error("The config key: {} is incorrect! The value is {}.", + DFS_ROUTER_ASYNC_RPC_NS_HANDLER_COUNT_KEY, nsHandlerInfo); + continue; + } + nsAsyncHandlerCount.put(nsHandlerItems[0], Integer.parseInt(nsHandlerItems[1])); + } + } + + private void initAsyncHandlerThreadPools4Ns(String nsId, int dedicatedHandlers) { + asyncRouterHandlerExecutors.computeIfAbsent(nsId, id -> Executors.newFixedThreadPool( + dedicatedHandlers, new AsyncThreadFactory("Router Async Handler for " + id + " #"))); + } + /** * Clear expired namespace in the shared RouterStateIdContext. */ @@ -555,7 +698,7 @@ BalanceProcedureScheduler getFedRenameScheduler() { * @return routerStateIdContext */ @VisibleForTesting - protected RouterStateIdContext getRouterStateIdContext() { + public RouterStateIdContext getRouterStateIdContext() { return routerStateIdContext; } @@ -636,7 +779,7 @@ public InetSocketAddress getRpcAddress() { * client requests. * @throws UnsupportedOperationException If the operation is not supported. */ - void checkOperation(OperationCategory op, boolean supported) + public void checkOperation(OperationCategory op, boolean supported) throws StandbyException, UnsupportedOperationException { checkOperation(op); @@ -658,7 +801,7 @@ void checkOperation(OperationCategory op, boolean supported) * @throws StandbyException If the Router is in safe mode and cannot serve * client requests. */ - void checkOperation(OperationCategory op) + public void checkOperation(OperationCategory op) throws StandbyException { // Log the function we are currently calling. if (rpcMonitor != null) { @@ -724,8 +867,9 @@ static String getMethodName() { * If the namespace is unavailable, retry with other namespaces. * @param expected return type. * @param method the remote method. + * @param clazz the type of return value. * @return the response received after invoking method. - * @throws IOException + * @throws IOException if there is no namespace available or other ioExceptions. */ T invokeAtAvailableNs(RemoteMethod method, Class clazz) throws IOException { @@ -751,6 +895,47 @@ T invokeAtAvailableNs(RemoteMethod method, Class clazz) return invokeOnNs(method, clazz, io, nss); } + /** + * Invokes the method at default namespace, if default namespace is not + * available then at the other available namespaces. + * If the namespace is unavailable, retry with other namespaces. + * Asynchronous version of invokeAtAvailableNs method. + * @param expected return type. + * @param method the remote method. + * @param clazz the type of return value. + * @return the response received after invoking method. + * @throws IOException if there is no namespace available or other ioExceptions. + */ + public T invokeAtAvailableNsAsync(RemoteMethod method, Class clazz) + throws IOException { + String nsId = subclusterResolver.getDefaultNamespace(); + // If default Ns is not present return result from first namespace. + Set nss = namenodeResolver.getNamespaces(); + // If no namespace is available, throw IOException. + IOException io = new IOException("No namespace available."); + + asyncComplete(null); + if (!nsId.isEmpty()) { + asyncTry(() -> { + getRPCClient().invokeSingle(nsId, method, clazz); + }); + + asyncCatch((AsyncCatchFunction)(res, ioe) -> { + if (!clientProto.isUnavailableSubclusterException(ioe)) { + LOG.debug("{} exception cannot be retried", + ioe.getClass().getSimpleName()); + throw ioe; + } + nss.removeIf(n -> n.getNameserviceId().equals(nsId)); + invokeOnNsAsync(method, clazz, io, nss); + }, IOException.class); + } else { + // If not have default NS. + invokeOnNsAsync(method, clazz, io, nss); + } + return asyncReturn(clazz); + } + /** * Invoke the method sequentially on available namespaces, * throw no namespace available exception, if no namespaces are available. @@ -759,7 +944,7 @@ T invokeAtAvailableNs(RemoteMethod method, Class clazz) * @param ioe IOException . * @param nss List of name spaces in the federation * @return the response received after invoking method. - * @throws IOException + * @throws IOException if there is no namespace available or other ioExceptions. */ T invokeOnNs(RemoteMethod method, Class clazz, IOException ioe, Set nss) throws IOException { @@ -784,6 +969,61 @@ T invokeOnNs(RemoteMethod method, Class clazz, IOException ioe, throw ioe; } + /** + * Invoke the method sequentially on available namespaces, + * throw no namespace available exception, if no namespaces are available. + * Asynchronous version of invokeOnNs method. + * @param method the remote method. + * @param clazz Class for the return type. + * @param ioe IOException . + * @param nss List of name spaces in the federation + * @return the response received after invoking method. + * @throws IOException if there is no namespace available or other ioExceptions. + */ + T invokeOnNsAsync(RemoteMethod method, Class clazz, IOException ioe, + Set nss) throws IOException { + if (nss.isEmpty()) { + throw ioe; + } + + asyncComplete(null); + Iterator nsIterator = nss.iterator(); + asyncForEach(nsIterator, (foreach, fnInfo) -> { + String nsId = fnInfo.getNameserviceId(); + LOG.debug("Invoking {} on namespace {}", method, nsId); + asyncTry(() -> { + getRPCClient().invokeSingle(nsId, method, clazz); + asyncApply(result -> { + if (result != null) { + foreach.breakNow(); + return result; + } + return null; + }); + }); + + asyncCatch((CatchFunction)(ret, ex) -> { + LOG.debug("Failed to invoke {} on namespace {}", method, nsId, ex); + // Ignore the exception and try on other namespace, if the tried + // namespace is unavailable, else throw the received exception. + if (!clientProto.isUnavailableSubclusterException(ex)) { + throw ex; + } + return null; + }, IOException.class); + }); + + asyncApply(obj -> { + if (obj == null) { + // Couldn't get a response from any of the namespace, throw ioe. + throw ioe; + } + return obj; + }); + + return asyncReturn(clazz); + } + @Override // ClientProtocol public Token getDelegationToken(Text renewer) throws IOException { @@ -835,6 +1075,10 @@ public HdfsFileStatus create(String src, FsPermission masked, */ RemoteLocation getCreateLocation(final String src) throws IOException { final List locations = getLocationsForPath(src, true); + if (isAsync()) { + getCreateLocationAsync(src, locations); + return asyncReturn(RemoteLocation.class); + } return getCreateLocation(src, locations); } @@ -871,6 +1115,44 @@ RemoteLocation getCreateLocation( return createLocation; } + /** + * Get the location to create a file. It checks if the file already existed + * in one of the locations. + * Asynchronous version of getCreateLocation method. + * + * @param src Path of the file to check. + * @param locations Prefetched locations for the file. + * @return The remote location for this file. + * @throws IOException If the file has no creation location. + */ + public 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); + if (locations.size() > 1) { + asyncTry(() -> { + getExistingLocationAsync(src, locations); + asyncApply((ApplyFunction) existingLocation -> { + if (existingLocation != null) { + LOG.debug("{} already exists in {}.", src, existingLocation); + return existingLocation; + } + return createLocation; + }); + }); + asyncCatch((o, e) -> createLocation, FileNotFoundException.class); + } else { + asyncComplete(createLocation); + } + + return asyncReturn(RemoteLocation.class); + } + /** * Gets the remote location where the file exists. * @param src the name of file. @@ -892,6 +1174,31 @@ private RemoteLocation getExistingLocation(String src, return null; } + /** + * Gets the remote location where the file exists. + * Asynchronous version of getExistingLocation method. + * @param src the name of file. + * @param locations all the remote locations. + * @return the remote location of the file if it exists, else null. + * @throws IOException in case of any exception. + */ + private RemoteLocation getExistingLocationAsync(String src, + List locations) throws IOException { + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + getRPCClient().invokeConcurrent( + locations, method, true, false, HdfsFileStatus.class); + asyncApply((ApplyFunction, Object>) results -> { + for (RemoteLocation loc : locations) { + if (results.get(loc) != null) { + return loc; + } + } + return null; + }); + return asyncReturn(RemoteLocation.class); + } + @Override // ClientProtocol public LastBlockWithStatus append(String src, final String clientName, final EnumSetWritable flag) throws IOException { @@ -1111,8 +1418,13 @@ private DatanodeInfo[] getCachedDatanodeReportImpl( try { DatanodeInfo[] dns = clientProto.getDatanodeReport(type); + if (router.getRpcServer().isAsync()) { + dns = syncReturn(DatanodeInfo[].class); + } LOG.debug("Refresh cached DN report with {} datanodes", dns.length); return dns; + } catch (Exception e) { + throw new IOException(e); } finally { // Reset ugi to remote user for remaining operations. RouterRpcServer.resetCurrentUser(); @@ -1146,6 +1458,38 @@ public DatanodeInfo[] getDatanodeReport( return toArray(datanodes, DatanodeInfo.class); } + /** + * Get the datanode report with a timeout. + * Asynchronous version of the getDatanodeReport method. + * @param type Type of the datanode. + * @param requireResponse If we require all the namespaces to report. + * @param timeOutMs Time out for the reply in milliseconds. + * @return List of datanodes. + * @throws IOException If it cannot get the report. + */ + 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(); + getRPCClient().invokeConcurrent(nss, method, requireResponse, false, + timeOutMs, DatanodeInfo[].class); + + asyncApply((ApplyFunction, + DatanodeInfo[]>) results -> { + updateDnMap(results, datanodesMap); + // Map -> Array + Collection datanodes = datanodesMap.values(); + return toArray(datanodes, DatanodeInfo.class); + }); + return asyncReturn(DatanodeInfo[].class); + } + @Override // ClientProtocol public DatanodeStorageReport[] getDatanodeStorageReport( DatanodeReportType type) throws IOException { @@ -1164,6 +1508,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. * @@ -1196,6 +1545,42 @@ public Map getDatanodeStorageReportMap( return ret; } + /** + * Get the list of datanodes per subcluster. + * Asynchronous version of getDatanodeStorageReportMap method. + * @param type Type of the datanodes to get. + * @param requireResponse If true an exception will be thrown if all calls do + * not complete. If false exceptions are ignored and all data results + * successfully received are returned. + * @param timeOutMs Time out for the reply in milliseconds. + * @return nsId to datanode list. + * @throws IOException If the method cannot be invoked remotely. + */ + public Map getDatanodeStorageReportMapAsync( + DatanodeReportType type, boolean requireResponse, long timeOutMs) + throws IOException { + + Map ret = new LinkedHashMap<>(); + RemoteMethod method = new RemoteMethod("getDatanodeStorageReport", + new Class[] {DatanodeReportType.class}, type); + Set nss = namenodeResolver.getNamespaces(); + getRPCClient().invokeConcurrent( + nss, method, requireResponse, false, timeOutMs, DatanodeStorageReport[].class); + + asyncApply((ApplyFunction, + Map>) results -> { + for (Entry entry : + results.entrySet()) { + FederationNamespaceInfo ns = entry.getKey(); + String nsId = ns.getNameserviceId(); + DatanodeStorageReport[] result = entry.getValue(); + ret.put(nsId, result); + } + return ret; + }); + return asyncReturn(ret.getClass()); + } + @Override // ClientProtocol public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException { @@ -1708,7 +2093,7 @@ public Long getNextSPSPath() throws IOException { * @return Prioritized list of locations in the federated cluster. * @throws IOException If the location for this path cannot be determined. */ - protected List getLocationsForPath(String path, + public List getLocationsForPath(String path, boolean failIfLocked) throws IOException { return getLocationsForPath(path, failIfLocked, true); } @@ -1723,7 +2108,7 @@ protected List getLocationsForPath(String path, * @return Prioritized list of locations in the federated cluster. * @throws IOException If the location for this path cannot be determined. */ - protected List getLocationsForPath(String path, + public List getLocationsForPath(String path, boolean failIfLocked, boolean needQuotaVerify) throws IOException { try { if (failIfLocked) { @@ -1844,7 +2229,7 @@ static void resetCurrentUser() { * @param clazz Class of the values. * @return Array with the outputs. */ - static T[] merge( + public static T[] merge( Map map, Class clazz) { // Put all results into a set to avoid repeats @@ -1905,7 +2290,7 @@ public FederationRPCMetrics getRPCMetrics() { * @param path Path to check. * @return If a path should be in all subclusters. */ - boolean isPathAll(final String path) { + public boolean isPathAll(final String path) { MountTable entry = getMountTable(path); return entry != null && entry.isAll(); } @@ -1940,9 +2325,9 @@ private MountTable getMountTable(final String path){ * mount entry. * @param path The path on which the operation need to be invoked. * @return true if the call is supposed to invoked on all locations. - * @throws IOException + * @throws IOException If an I/O error occurs. */ - boolean isInvokeConcurrent(final String path) throws IOException { + public boolean isInvokeConcurrent(final String path) throws IOException { if (subclusterResolver instanceof MountTableResolver) { MountTableResolver mountTableResolver = (MountTableResolver) subclusterResolver; @@ -2011,6 +2396,37 @@ public DatanodeInfo[] getSlowDatanodeReport(boolean requireResponse, long timeOu return toArray(datanodes, DatanodeInfo.class); } + /** + * Get the slow running datanodes report with a timeout. + * Asynchronous version of the getSlowDatanodeReport method. + * + * @param requireResponse If we require all the namespaces to report. + * @param timeOutMs Time out for the reply in milliseconds. + * @return List of datanodes. + * @throws IOException If it cannot get the report. + */ + 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(); + getRPCClient().invokeConcurrent(nss, method, requireResponse, false, + timeOutMs, DatanodeInfo[].class); + + asyncApply((ApplyFunction, + DatanodeInfo[]>) results -> { + updateDnMap(results, datanodesMap); + // Map -> Array + Collection datanodes = datanodesMap.values(); + return toArray(datanodes, DatanodeInfo.class); + }); + + return asyncReturn(DatanodeInfo[].class); + } + private void updateDnMap(Map results, Map datanodesMap) { for (Entry entry : @@ -2068,4 +2484,32 @@ public ListenableFuture reload( return executorService.submit(() -> load(type)); } } + + public boolean isAsync() { + return this.enableAsync; + } + + public Map getAsyncRouterHandlerExecutors() { + return asyncRouterHandlerExecutors; + } + + public ExecutorService getRouterAsyncHandlerDefaultExecutor() { + return routerDefaultAsyncHandlerExecutor; + } + + private static class AsyncThreadFactory implements ThreadFactory { + private final String namePrefix; + private final AtomicInteger threadNumber = new AtomicInteger(1); + + AsyncThreadFactory(String namePrefix) { + this.namePrefix = namePrefix; + } + + @Override + public Thread newThread(@NonNull Runnable r) { + Thread thread = new Thread(r, namePrefix + threadNumber.getAndIncrement()); + thread.setDaemon(true); + return thread; + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java index e239e5e9059ff..21e3f16f20613 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateIdContext.java @@ -48,7 +48,7 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -class RouterStateIdContext implements AlignmentContext { +public class RouterStateIdContext implements AlignmentContext { private final HashSet coordinatedMethods; /** @@ -93,6 +93,8 @@ class RouterStateIdContext implements AlignmentContext { /** * Adds the {@link #namespaceIdMap} to the response header that will be sent to a client. + * + * @param headerBuilder the response header that will be sent to a client. */ public void setResponseHeaderState(RpcResponseHeaderProto.Builder headerBuilder) { if (namespaceIdMap.isEmpty()) { @@ -110,7 +112,8 @@ public void setResponseHeaderState(RpcResponseHeaderProto.Builder headerBuilder) } public LongAccumulator getNamespaceStateId(String nsId) { - return namespaceIdMap.computeIfAbsent(nsId, key -> new LongAccumulator(Math::max, Long.MIN_VALUE)); + return namespaceIdMap.computeIfAbsent(nsId, + key -> new LongAccumulator(Math::max, Long.MIN_VALUE)); } public List getNamespaces() { @@ -127,6 +130,9 @@ public void removeNamespaceStateId(String nsId) { /** * Utility function to parse routerFederatedState field in RPC headers. + * + * @param byteString the byte string of routerFederatedState. + * @return the router federated state map. */ public static Map getRouterFederatedStateMap(ByteString byteString) { if (byteString != null) { @@ -148,7 +154,8 @@ public static long getClientStateIdFromCurrentCall(String nsId) { if (call != null) { ByteString callFederatedNamespaceState = call.getFederatedNamespaceState(); if (callFederatedNamespaceState != null) { - Map clientFederatedStateIds = getRouterFederatedStateMap(callFederatedNamespaceState); + Map clientFederatedStateIds = + getRouterFederatedStateMap(callFederatedNamespaceState); clientStateID = clientFederatedStateIds.getOrDefault(nsId, Long.MIN_VALUE); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ThreadLocalContext.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ThreadLocalContext.java new file mode 100644 index 0000000000000..6faef079bd19a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ThreadLocalContext.java @@ -0,0 +1,101 @@ +/** + * 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.server.federation.metrics.FederationRPCPerformanceMonitor; +import org.apache.hadoop.ipc.CallerContext; +import org.apache.hadoop.ipc.Server; + +/** + * The ThreadLocalContext class is designed to capture and transfer the context of a + * thread-local environment within the Hadoop Distributed File System (HDFS) federation + * router operations. This is particularly useful for preserving the state across + * asynchronous operations where the context needs to be maintained consistently. + * + * The context includes details such as the current call being processed by the server, the + * caller's context, and performance monitoring timestamps. By transferring this context, + * the class ensures that the operations performed on worker threads correctly reflect + * the state of the original calling thread. + * + * Here is a high-level overview of the main components captured by this context: + *

    + *
  • {@link Server.Call} - Represents the current server call.
  • + *
  • {@link CallerContext} - Stores information about the caller.
  • + *
  • startOpTime - Time for an operation to be received in the Router.
  • + *
  • proxyOpTime - Time for an operation to be sent to the Namenode.
  • + *
+ * + * This class is typically used in scenarios where asynchronous processing is involved, to + * ensure that the thread executing the asynchronous task has the correct context applied. + * + * @see Server + * @see CallerContext + * @see FederationRPCPerformanceMonitor + */ +public class ThreadLocalContext { + + /** The current server call being processed. */ + private final Server.Call call; + /** The caller context containing information about the caller. */ + private final CallerContext context; + /** Time for an operation to be received in the Router. */ + private final long startOpTime; + /** Time for an operation to be sent to the Namenode. */ + private final long proxyOpTime; + + /** + * Constructs a new {@link ThreadLocalContext} instance, capturing the current + * thread-local context at the point of creation. + */ + public ThreadLocalContext() { + this.call = Server.getCurCall().get(); + this.context = CallerContext.getCurrent(); + this.startOpTime = FederationRPCPerformanceMonitor.getStartOpTime(); + this.proxyOpTime = FederationRPCPerformanceMonitor.getProxyOpTime(); + } + + /** + * Transfers the captured context to the current thread. This method is used to apply + * the context to worker threads that are processing asynchronous tasks, ensuring + * that the task execution reflects the state of the original calling thread. + */ + public void transfer() { + if (call != null) { + Server.getCurCall().set(call); + } + if (context != null) { + CallerContext.setCurrent(context); + } + if (startOpTime != -1L) { + FederationRPCPerformanceMonitor.setStartOpTime(startOpTime); + } + if (proxyOpTime != -1L) { + FederationRPCPerformanceMonitor.setProxyOpTime(proxyOpTime); + } + } + + @Override + public String toString() { + return "ThreadLocalContext{" + + "call=" + call + + ", context=" + context + + ", startOpTime=" + startOpTime + + ", proxyOpTime=" + proxyOpTime + + '}'; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncErasureCoding.java new file mode 100644 index 0000000000000..d40c6a1a93afc --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncErasureCoding.java @@ -0,0 +1,255 @@ +/** + * 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.async; + +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.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.federation.router.ErasureCoding; +import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RemoteParam; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.merge; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; + +/** + * Provides asynchronous operations for erasure coding in HDFS Federation. + * This class extends {@link org.apache.hadoop.hdfs.server.federation.router.ErasureCoding} + * and overrides its methods to perform erasure coding operations in a non-blocking manner, + * allowing for concurrent execution and improved performance. + */ +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(); + } + + /** + * Asynchronously get an array of all erasure coding policies. + * This method checks the operation category and then invokes the + * getErasureCodingPolicies method concurrently across all namespaces. + *

+ * The results are merged and returned as an array of ErasureCodingPolicyInfo. + * + * @return Array of ErasureCodingPolicyInfo. + * @throws IOException If an I/O error occurs. + */ + @Override + public ErasureCodingPolicyInfo[] getErasureCodingPolicies() + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getErasureCodingPolicies"); + Set nss = namenodeResolver.getNamespaces(); + + rpcClient.invokeConcurrent( + nss, method, true, false, ErasureCodingPolicyInfo[].class); + asyncApply( + (ApplyFunction, + ErasureCodingPolicyInfo[]>) ret -> merge(ret, ErasureCodingPolicyInfo.class)); + + return asyncReturn(ErasureCodingPolicyInfo[].class); + } + + /** + * Asynchronously get the erasure coding codecs available. + * This method checks the operation category and then invokes the + * getErasureCodingCodecs method concurrently across all namespaces. + *

+ * The results are merged into a single map of codec names to codec properties. + * + * @return Map of erasure coding codecs. + * @throws IOException If an I/O error occurs. + */ + @Override + public Map getErasureCodingCodecs() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getErasureCodingCodecs"); + Set nss = namenodeResolver.getNamespaces(); + + rpcClient.invokeConcurrent( + nss, method, true, false, Map.class); + + asyncApply((ApplyFunction>, Map>) 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; + }); + + return asyncReturn(Map.class); + } + + /** + * Asynchronously add an array of erasure coding policies. + * This method checks the operation category and then invokes the + * addErasureCodingPolicies method concurrently across all namespaces. + *

+ * The results are merged and returned as an array of AddErasureCodingPolicyResponse. + * + * @param policies Array of erasure coding policies to add. + * @return Array of AddErasureCodingPolicyResponse. + * @throws IOException If an I/O error occurs. + */ + @Override + 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(); + + rpcClient.invokeConcurrent( + nss, method, true, false, AddErasureCodingPolicyResponse[].class); + + asyncApply( + (ApplyFunction, + AddErasureCodingPolicyResponse[]>) ret -> { + return merge(ret, AddErasureCodingPolicyResponse.class); + }); + return asyncReturn(AddErasureCodingPolicyResponse[].class); + } + + /** + * Asynchronously get the erasure coding policy for a given source path. + * This method checks the operation category and then invokes the + * getErasureCodingPolicy method sequentially for the given path. + *

+ * The result is returned as an ErasureCodingPolicy object. + * + * @param src Source path to get the erasure coding policy for. + * @return ErasureCodingPolicy for the given path. + * @throws IOException If an I/O error occurs. + */ + @Override + public ErasureCodingPolicy getErasureCodingPolicy(String src) + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + final List locations = + rpcServer.getLocationsForPath(src, false, false); + RemoteMethod remoteMethod = new RemoteMethod("getErasureCodingPolicy", + new Class[] {String.class}, new RemoteParam()); + rpcClient.invokeSequential( + locations, remoteMethod, null, null); + + asyncApply(ret -> { + return (ErasureCodingPolicy) ret; + }); + + return asyncReturn(ErasureCodingPolicy.class); + } + + /** + * Asynchronously get the EC topology result for the given policies. + * This method checks the operation category and then invokes the + * getECTopologyResultForPolicies method concurrently across all namespaces. + *

+ * The results are merged and the first unsupported result is returned. + * + * @param policyNames Array of policy names to check. + * @return ECTopologyVerifierResult for the policies. + * @throws IOException If an I/O error occurs. + */ + @Override + 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."); + } + + rpcClient.invokeConcurrent(nss, method, true, false, + ECTopologyVerifierResult.class); + asyncApply((ApplyFunction, + ECTopologyVerifierResult>) 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()); + }); + return asyncReturn(ECTopologyVerifierResult.class); + } + + /** + * Asynchronously get the erasure coding block group statistics. + * This method checks the operation category and then invokes the + * getECBlockGroupStats method concurrently across all namespaces. + *

+ * The results are merged and returned as an ECBlockGroupStats object. + * + * @return ECBlockGroupStats for the erasure coding block groups. + * @throws IOException If an I/O error occurs. + */ + @Override + public ECBlockGroupStats getECBlockGroupStats() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getECBlockGroupStats"); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent( + nss, method, true, false, ECBlockGroupStats.class); + + asyncApply((ApplyFunction, + ECBlockGroupStats>) allStats -> { + return ECBlockGroupStats.merge(allStats.values()); + }); + return asyncReturn(ECBlockGroupStats.class); + } +} \ 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/async/AsyncQuota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncQuota.java new file mode 100644 index 0000000000000..0980cf2a2fbce --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncQuota.java @@ -0,0 +1,101 @@ +/** + * 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.async; + +import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.federation.router.Quota; +import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RemoteParam; +import org.apache.hadoop.hdfs.server.federation.router.Router; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletionException; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; + +/** + * Provides asynchronous operations for managing quotas in HDFS Federation. + * This class extends {@link org.apache.hadoop.hdfs.server.federation.router.Quota} + * and overrides its methods to perform quota operations in a non-blocking manner, + * allowing for concurrent execution and improved performance. + */ +public class AsyncQuota extends Quota { + + /** RPC server to receive client calls. */ + private final RouterRpcServer rpcServer; + /** RPC clients to connect to the Namenodes. */ + private final RouterRpcClient rpcClient; + private final Router router; + + public AsyncQuota(Router router, RouterRpcServer server) { + super(router, server); + this.router = router; + this.rpcServer = server; + this.rpcClient = this.rpcServer.getRPCClient(); + } + + /** + * Async get aggregated quota usage for the federation path. + * @param path Federation path. + * @return Aggregated quota. + * @throws IOException If the quota system is disabled. + */ + @Override + public QuotaUsage getQuotaUsage(String path) throws IOException { + getEachQuotaUsage(path); + + asyncApply(o -> { + Map results = (Map) o; + try { + return aggregateQuota(path, results); + } catch (IOException e) { + throw new CompletionException(e); + } + }); + return asyncReturn(QuotaUsage.class); + } + + /** + * Get quota usage for the federation path. + * @param path Federation path. + * @return quota usage for each remote location. + * @throws IOException If the quota system is disabled. + */ + @Override + protected Map getEachQuotaUsage(String path) + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + if (!router.isQuotaEnabled()) { + throw new IOException("The quota system is disabled in Router."); + } + + final List quotaLocs = getValidQuotaLocations(path); + RemoteMethod method = new RemoteMethod("getQuotaUsage", + new Class[] {String.class}, new RemoteParam()); + rpcClient.invokeConcurrent( + quotaLocs, method, true, false, QuotaUsage.class); + return asyncReturn(Map.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncCacheAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncCacheAdmin.java new file mode 100644 index 0000000000000..20ec36c935cf4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncCacheAdmin.java @@ -0,0 +1,109 @@ +/** + * 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.async; + +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.fs.BatchedRemoteIterator.BatchedEntries; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.federation.router.RouterCacheAdmin; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.Map; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; + +/** + * Module that implements all the asynchronous RPC calls in + * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol} related to Cache Admin + * in the {@link RouterRpcServer}. + */ +public class RouterAsyncCacheAdmin extends RouterCacheAdmin { + + public RouterAsyncCacheAdmin(RouterRpcServer server) { + super(server); + } + + /** + * Asynchronously adds a new cache directive with the given path and flags. + * This method invokes the addCacheDirective method concurrently across all + * namespaces, and returns the first response as a long value representing the + * directive ID. + * + * @param path The cache directive path. + * @param flags The cache flags. + * @return The ID of the newly added cache directive. + * @throws IOException If an I/O error occurs. + */ + @Override + public long addCacheDirective( + CacheDirectiveInfo path, EnumSet flags) throws IOException { + invokeAddCacheDirective(path, flags); + asyncApply((ApplyFunction, Long>) + response -> response.values().iterator().next()); + return asyncReturn(Long.class); + } + + /** + * Asynchronously lists cache directives based on the provided previous ID and filter. + * This method invokes the listCacheDirectives method concurrently across all + * namespaces, and returns the first response as a BatchedEntries object containing + * the cache directive entries. + * + * @param prevId The previous ID from which to start listing. + * @param filter The filter to apply to the cache directives. + * @return BatchedEntries of cache directive entries. + * @throws IOException If an I/O error occurs. + */ + @Override + public BatchedEntries listCacheDirectives( + long prevId, CacheDirectiveInfo filter) throws IOException { + invokeListCacheDirectives(prevId, filter); + asyncApply((ApplyFunction>) + response -> (BatchedEntries) response.values().iterator().next()); + return asyncReturn(BatchedEntries.class); + } + + /** + * Asynchronously lists cache pools starting from the provided key. + * This method invokes the listCachePools method concurrently across all namespaces, + * and returns the first response as a BatchedEntries object containing the cache + * pool entries. + * + * @param prevKey The previous key from which to start listing. + * @return BatchedEntries of cache pool entries. + * @throws IOException If an I/O error occurs. + */ + @Override + public BatchedEntries listCachePools(String prevKey) throws IOException { + invokeListCachePools(prevKey); + asyncApply((ApplyFunction, + BatchedEntries>) + results -> results.values().iterator().next()); + return asyncReturn(BatchedEntries.class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncClientProtocol.java new file mode 100644 index 0000000000000..f991b27b029e4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncClientProtocol.java @@ -0,0 +1,1089 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.crypto.CryptoProtocolVersion; +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.permission.FsPermission; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.protocol.ClientProtocol; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +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.ReplicatedBlockStats; +import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; +import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; +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.router.NoLocationException; +import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RemoteParam; +import org.apache.hadoop.hdfs.server.federation.router.RemoteResult; +import org.apache.hadoop.hdfs.server.federation.router.RouterClientProtocol; +import org.apache.hadoop.hdfs.server.federation.router.RouterFederationRename; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncApplyFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncCatchFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.CatchFunction; +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.security.UserGroupInformation; +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.Arrays; +import java.util.Collection; +import java.util.Collections; +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.stream.Collectors; + +import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.updateMountPointStatus; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCatch; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCompleteWith; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncForEach; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncTry; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.getCompletableFuture; + +/** + * Module that implements all the async RPC calls in {@link ClientProtocol} in the + * {@link RouterRpcServer}. + */ +public class RouterAsyncClientProtocol extends RouterClientProtocol { + private static final Logger LOG = + LoggerFactory.getLogger(RouterAsyncClientProtocol.class.getName()); + + private final RouterRpcServer rpcServer; + private final RouterRpcClient rpcClient; + private final RouterFederationRename rbfRename; + private final FileSubclusterResolver subclusterResolver; + private final ActiveNamenodeResolver namenodeResolver; + /** If it requires response from all subclusters. */ + private final boolean allowPartialList; + /** Time out when getting the mount statistics. */ + private long mountStatusTimeOut; + /** Identifier for the super user. */ + private String superUser; + /** Identifier for the super group. */ + private final String superGroup; + /** + * Caching server defaults so as to prevent redundant calls to namenode, + * similar to DFSClient, caching saves efforts when router connects + * to multiple clients. + */ + private volatile FsServerDefaults serverDefaults; + + public RouterAsyncClientProtocol(Configuration conf, RouterRpcServer rpcServer) { + super(conf, rpcServer); + this.rpcServer = rpcServer; + this.rpcClient = rpcServer.getRPCClient(); + this.rbfRename = getRbfRename(); + this.subclusterResolver = getSubclusterResolver(); + this.namenodeResolver = getNamenodeResolver(); + this.allowPartialList = isAllowPartialList(); + this.mountStatusTimeOut = getMountStatusTimeOut(); + this.superUser = getSuperUser(); + this.superGroup = getSuperGroup(); + } + + @Override + public FsServerDefaults getServerDefaults() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + long now = Time.monotonicNow(); + if ((serverDefaults == null) || (now - getServerDefaultsLastUpdate() + > getServerDefaultsValidityPeriod())) { + RemoteMethod method = new RemoteMethod("getServerDefaults"); + rpcServer.invokeAtAvailableNsAsync(method, FsServerDefaults.class); + asyncApply(o -> { + serverDefaults = (FsServerDefaults) o; + setServerDefaultsLastUpdate(now); + return serverDefaults; + }); + } else { + asyncComplete(serverDefaults); + } + 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 { + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + 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); + asyncApply((ApplyFunction) success -> { + if (!success) { + // This shouldn't happen as mkdirs returns true or exception + LOG.error("Couldn't create parents for {}", src); + } + return success; + }); + } + + 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 List locations = + rpcServer.getLocationsForPath(src, true); + final RemoteLocation[] createLocation = new RemoteLocation[1]; + asyncTry(() -> { + rpcServer.getCreateLocationAsync(src, locations); + asyncApply((AsyncApplyFunction) remoteLocation -> { + createLocation[0] = remoteLocation; + rpcClient.invokeSingle(remoteLocation, method, HdfsFileStatus.class); + asyncApply((ApplyFunction) status -> { + status.setNamespace(remoteLocation.getNameserviceId()); + return status; + }); + }); + }); + asyncCatch((AsyncCatchFunction) (o, ioe) -> { + final List newLocations = checkFaultTolerantRetry( + method, src, ioe, createLocation[0], locations); + rpcClient.invokeSequential( + newLocations, method, HdfsFileStatus.class, null); + }, IOException.class); + + return asyncReturn(HdfsFileStatus.class); + } + + @Override + public LastBlockWithStatus append( + String src, String clientName, + EnumSetWritable flag) throws IOException { + 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); + asyncApply((ApplyFunction) result -> { + LastBlockWithStatus lbws = (LastBlockWithStatus) result.getResult(); + lbws.getFileStatus().setNamespace(result.getLocation().getNameserviceId()); + return lbws; + }); + return asyncReturn(LastBlockWithStatus.class); + } + + @Deprecated + @Override + public boolean rename(final String src, final String dst) + throws IOException { + 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()) { + asyncComplete( + rbfRename.routerFedRename(src, dst, srcLocations, dstLocations)); + return asyncReturn(Boolean.class); + } + RemoteMethod method = new RemoteMethod("rename", + new Class[] {String.class, String.class}, + new RemoteParam(), dstParam); + isMultiDestDirectory(src); + asyncApply((AsyncApplyFunction) isMultiDestDirectory -> { + if (isMultiDestDirectory) { + if (locs.size() != srcLocations.size()) { + throw new IOException("Rename of " + src + " to " + dst + " is not" + + " allowed. The number of remote locations for both source and" + + " target should be same."); + } + rpcClient.invokeAll(locs, method); + } else { + rpcClient.invokeSequential(locs, method, Boolean.class, + Boolean.TRUE); + } + }); + return asyncReturn(Boolean.class); + } + + @Override + public void rename2( + final String src, final String dst, + final Options.Rename... options) throws IOException { + 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); + asyncApply((AsyncApplyFunction) isMultiDestDirectory -> { + if (isMultiDestDirectory) { + if (locs.size() != srcLocations.size()) { + throw new IOException("Rename of " + src + " to " + dst + " is not" + + " allowed. The number of remote locations for both source and" + + " target should be same."); + } + rpcClient.invokeConcurrent(locs, method); + } else { + rpcClient.invokeSequential(locs, method, null, null); + } + }); + } + + @Override + public void concat(String trg, String[] src) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.WRITE); + + // Concat only effects when all files in the same namespace. + getFileRemoteLocation(trg); + asyncApply((AsyncApplyFunction) targetDestination -> { + if (targetDestination == null) { + throw new IOException("Cannot find target file - " + trg); + } + String targetNameService = targetDestination.getNameserviceId(); + String[] sourceDestinations = new String[src.length]; + int[] index = new int[1]; + asyncForEach(Arrays.stream(src).iterator(), (forEachRun, sourceFile) -> { + getFileRemoteLocation(sourceFile); + asyncApply((ApplyFunction) srcLocation -> { + if (srcLocation == null) { + throw new IOException("Cannot find source file - " + sourceFile); + } + sourceDestinations[index[0]++] = srcLocation.getDest(); + if (!targetNameService.equals(srcLocation.getNameserviceId())) { + throw new IOException("Cannot concatenate source file " + sourceFile + + " because it is located in a different namespace" + " with nameservice " + + srcLocation.getNameserviceId() + " from the target file with nameservice " + + targetNameService); + } + return null; + }); + }); + asyncApply((AsyncApplyFunction) o -> { + // Invoke + RemoteMethod method = new RemoteMethod("concat", + new Class[] {String.class, String[].class}, + targetDestination.getDest(), sourceDestinations); + rpcClient.invokeSingle(targetDestination, method, Void.class); + }); + }); + } + + @Override + public boolean mkdirs(String src, FsPermission masked, boolean createParent) + throws IOException { + 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); + } + + asyncComplete(false); + if (locations.size() > 1) { + // Check if this directory already exists + asyncTry(() -> { + getFileInfo(src); + asyncApply((ApplyFunction) fileStatus -> { + if (fileStatus != null) { + // When existing, the NN doesn't return an exception; return true + return true; + } + return false; + }); + }); + asyncCatch((ret, ex) -> { + // Can't query if this file exists or not. + LOG.error("Error getting file info for {} while proxying mkdirs: {}", + src, ex.getMessage()); + return false; + }, IOException.class); + } + + final RemoteLocation firstLocation = locations.get(0); + asyncApply((AsyncApplyFunction) success -> { + if (success) { + asyncComplete(true); + return; + } + asyncTry(() -> { + rpcClient.invokeSingle(firstLocation, method, Boolean.class); + }); + + asyncCatch((AsyncCatchFunction) (o, ioe) -> { + final List newLocations = checkFaultTolerantRetry( + method, src, ioe, firstLocation, locations); + rpcClient.invokeSequential( + newLocations, method, Boolean.class, Boolean.TRUE); + }, IOException.class); + }); + + return asyncReturn(Boolean.class); + } + + @Override + public DirectoryListing getListing( + String src, byte[] startAfter, boolean needLocation) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + GetListingComparator comparator = RouterClientProtocol.getComparator(); + getListingInt(src, startAfter, needLocation); + asyncApply((AsyncApplyFunction>, Object>) + listings -> { + TreeMap nnListing = new TreeMap<>(comparator); + int totalRemainingEntries = 0; + final int[] remainingEntries = {0}; + boolean namenodeListingExists = false; + // Check the subcluster listing with the smallest name to make sure + // no file is skipped across subclusters + byte[] lastName = null; + 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 (!allowPartialList) { + throw 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[0]++; + } else { + nnListing.put(filename, file); + } + } + remainingEntries[0] += listing.getRemainingEntries(); + } + } + } + + // 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); + byte[] finalLastName = lastName; + asyncForEach(children.iterator(), (forEachRun, child) -> { + long date = 0; + if (dates != null && dates.containsKey(child)) { + date = dates.get(child); + } + Path childPath = new Path(src, child); + getMountPointStatus(childPath.toString(), 0, date); + asyncApply((ApplyFunction) dirStatus -> { + // if there is no subcluster path, always add mount point + byte[] bChild = DFSUtil.string2Bytes(child); + if (finalLastName == null) { + nnListing.put(bChild, dirStatus); + } else { + if (shouldAddMountPoint(bChild, + finalLastName, startAfter, remainingEntries[0])) { + // This may overwrite existing listing entries with the mount point + // TODO don't add if already there? + nnListing.put(bChild, dirStatus); + } + } + return null; + }); + }); + asyncApply(o -> { + // 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[0] += (children.size() - i); + break; + } + } + } + return null; + }); + } + asyncComplete(namenodeListingExists); + asyncApply((ApplyFunction) exists -> { + if (!exists && 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[0]); + }); + }); + return asyncReturn(DirectoryListing.class); + } + + /** + * Get listing on remote locations. + */ + @Override + protected List> getListingInt( + String src, byte[] startAfter, boolean needLocation) throws IOException { + List locations = + rpcServer.getLocationsForPath(src, false, false); + // Locate the dir and fetch the listing. + if (locations.isEmpty()) { + asyncComplete(new ArrayList<>()); + return asyncReturn(List.class); + } + asyncTry(() -> { + 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); + }); + asyncCatch((CatchFunction) (o, e) -> { + LOG.debug("Cannot get locations for {}, {}.", src, e.getMessage()); + LOG.info("Cannot get locations for {}, {}.", src, e.getMessage()); + return new ArrayList<>(); + }, RouterResolveException.class); + return asyncReturn(List.class); + } + + + @Override + public HdfsFileStatus getFileInfo(String src) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + final IOException[] noLocationException = new IOException[1]; + asyncTry(() -> { + 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); + } + }); + asyncCatch((o, e) -> { + if (e instanceof NoLocationException + || e instanceof RouterResolveException) { + noLocationException[0] = e; + } + throw e; + }, IOException.class); + + asyncApply((AsyncApplyFunction) ret -> { + // If there is no real path, check mount points + if (ret == null) { + List children = subclusterResolver.getMountPoints(src); + if (children != null && !children.isEmpty()) { + Map dates = getMountPointDates(src); + long date = 0; + if (dates != null && dates.containsKey(src)) { + date = dates.get(src); + } + getMountPointStatus(src, children.size(), date, false); + } else if (children != null) { + // The src is a mount point, but there are no files or directories + getMountPointStatus(src, 0, 0, false); + } else { + asyncComplete(null); + } + asyncApply((ApplyFunction) result -> { + // Can't find mount point for path and the path didn't contain any sub monit points, + // throw the NoLocationException to client. + if (result == null && noLocationException[0] != null) { + throw noLocationException[0]; + } + + return result; + }); + } else { + asyncComplete(ret); + } + }); + + return asyncReturn(HdfsFileStatus.class); + } + + @Override + public RemoteLocation getFileRemoteLocation(String path) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + final List locations = rpcServer.getLocationsForPath(path, false, false); + if (locations.size() == 1) { + asyncComplete(locations.get(0)); + return asyncReturn(RemoteLocation.class); + } + + asyncForEach(locations.iterator(), (forEachRun, location) -> { + RemoteMethod method = + new RemoteMethod("getFileInfo", new Class[] {String.class}, new RemoteParam()); + rpcClient.invokeSequential(Collections.singletonList(location), method, + HdfsFileStatus.class, null); + asyncApply((ApplyFunction) ret -> { + if (ret != null) { + forEachRun.breakNow(); + return location; + } + return null; + }); + }); + + return asyncReturn(RemoteLocation.class); + } + + @Override + public HdfsFileStatus getMountPointStatus( + String name, int childrenNum, long date, boolean setPath) { + long modTime = date; + long accessTime = date; + final FsPermission[] permission = new FsPermission[]{FsPermission.getDirDefault()}; + final String[] owner = new String[]{this.superUser}; + final String[] group = new String[]{this.superGroup}; + final int[] childrenNums = new int[]{childrenNum}; + final EnumSet[] flags = + new EnumSet[]{EnumSet.noneOf(HdfsFileStatus.Flags.class)}; + asyncComplete(null); + if (getSubclusterResolver() instanceof MountTableResolver) { + asyncTry(() -> { + String mName = name.startsWith("/") ? name : "/" + name; + MountTableResolver mountTable = (MountTableResolver) subclusterResolver; + MountTable entry = mountTable.getMountPoint(mName); + if (entry != null) { + permission[0] = entry.getMode(); + owner[0] = entry.getOwnerName(); + group[0] = entry.getGroupName(); + + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + getFileInfoAll( + entry.getDestinations(), method, mountStatusTimeOut); + asyncApply((ApplyFunction) fInfo -> { + if (fInfo != null) { + permission[0] = fInfo.getPermission(); + owner[0] = fInfo.getOwner(); + group[0] = fInfo.getGroup(); + childrenNums[0] = fInfo.getChildrenNum(); + flags[0] = DFSUtil + .getFlags(fInfo.isEncrypted(), fInfo.isErasureCoded(), + fInfo.isSnapshotEnabled(), fInfo.hasAcl()); + } + return fInfo; + }); + } + }); + asyncCatch((CatchFunction) (status, e) -> { + LOG.error("Cannot get mount point: {}", e.getMessage()); + return status; + }, IOException.class); + } else { + try { + UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + owner[0] = ugi.getUserName(); + group[0] = ugi.getPrimaryGroupName(); + } catch (IOException e) { + String msg = "Cannot get remote user: " + e.getMessage(); + if (UserGroupInformation.isSecurityEnabled()) { + LOG.error(msg); + } else { + LOG.debug(msg); + } + } + } + long inodeId = 0; + HdfsFileStatus.Builder builder = new HdfsFileStatus.Builder(); + asyncApply((ApplyFunction) status -> { + 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(permission[0]) + .owner(owner[0]) + .group(group[0]) + .symlink(new byte[0]) + .fileId(inodeId) + .children(childrenNums[0]) + .flags(flags[0]) + .build(); + }); + return asyncReturn(HdfsFileStatus.class); + } + + @Override + protected HdfsFileStatus getFileInfoAll(final List locations, + final RemoteMethod method, long timeOutMs) throws IOException { + + asyncComplete(null); + // Get the file info from everybody + rpcClient.invokeConcurrent(locations, method, false, false, timeOutMs, + HdfsFileStatus.class); + asyncApply(res -> { + Map results = (Map) res; + 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; + }); + return asyncReturn(HdfsFileStatus.class); + } + + @Override + public boolean recoverLease(String src, String clientName) throws IOException { + super.recoverLease(src, clientName); + return asyncReturn(boolean.class); + } + + @Override + public long[] getStats() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + RemoteMethod method = new RemoteMethod("getStats"); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, false, long[].class); + asyncApply(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; + }); + return asyncReturn(long[].class); + } + + @Override + public ReplicatedBlockStats getReplicatedBlockStats() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getReplicatedBlockStats"); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent(nss, method, true, + false, ReplicatedBlockStats.class); + asyncApply(o -> { + Map ret = + (Map) o; + return ReplicatedBlockStats.merge(ret.values()); + }); + return asyncReturn(ReplicatedBlockStats.class); + } + + @Override + public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type) + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + return rpcServer.getDatanodeReportAsync(type, true, 0); + } + + @Override + public DatanodeInfo[] getSlowDatanodeReport() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + return rpcServer.getSlowDatanodeReportAsync(true, 0); + } + + @Override + public DatanodeStorageReport[] getDatanodeStorageReport( + HdfsConstants.DatanodeReportType type) throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + rpcServer.getDatanodeStorageReportMapAsync(type); + asyncApply((ApplyFunction< Map, DatanodeStorageReport[]>) + dnSubcluster -> mergeDtanodeStorageReport(dnSubcluster)); + return asyncReturn(DatanodeStorageReport[].class); + } + + public DatanodeStorageReport[] getDatanodeStorageReport( + HdfsConstants.DatanodeReportType type, boolean requireResponse, long timeOutMs) + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.UNCHECKED); + + rpcServer.getDatanodeStorageReportMapAsync(type, requireResponse, timeOutMs); + asyncApply((ApplyFunction< Map, DatanodeStorageReport[]>) + dnSubcluster -> mergeDtanodeStorageReport(dnSubcluster)); + return asyncReturn(DatanodeStorageReport[].class); + } + + @Override + public boolean setSafeMode(HdfsConstants.SafeModeAction action, + boolean isChecked) throws IOException { + 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); + + asyncApply(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(); + }); + return asyncReturn(Boolean.class); + } + + @Override + public boolean saveNamespace(long timeWindow, long txGap) throws IOException { + 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); + + asyncApply(o -> { + Map ret = + (Map) o; + boolean success = true; + for (boolean s : ret.values()) { + if (!s) { + success = false; + break; + } + } + return success; + }); + return asyncReturn(Boolean.class); + } + + @Override + public long rollEdits() throws IOException { + 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); + asyncApply(o -> { + Map ret = + (Map) o; + // Return the maximum txid + long txid = 0; + for (long t : ret.values()) { + if (t > txid) { + txid = t; + } + } + return txid; + }); + return asyncReturn(long.class); + } + + @Override + public boolean restoreFailedStorage(String arg) throws IOException { + 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); + asyncApply(o -> { + Map ret = + (Map) o; + boolean success = true; + for (boolean s : ret.values()) { + if (!s) { + success = false; + break; + } + } + return success; + }); + return asyncReturn(boolean.class); + } + + @Override + public RollingUpgradeInfo rollingUpgrade(HdfsConstants.RollingUpgradeAction action) + throws IOException { + 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); + asyncApply(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; + }); + return asyncReturn(RollingUpgradeInfo.class); + } + + @Override + public ContentSummary getContentSummary(String path) throws IOException { + 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); + + asyncApply(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 (!allowPartialList) { + throw 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 notFoundException; + } + return aggregateContentSummary(summaries); + }); + + return asyncReturn(ContentSummary.class); + } + + @Override + public long getCurrentEditLogTxid() throws IOException { + 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); + + asyncApply(o -> { + Map ret = + (Map) o; + // Return the maximum txid + long txid = 0; + for (long t : ret.values()) { + if (t > txid) { + txid = t; + } + } + return txid; + }); + return asyncReturn(long.class); + } + + @Override + public void msync() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ, true); + // Only msync to nameservices with observer reads enabled. + Set allNamespaces = namenodeResolver.getNamespaces(); + RemoteMethod method = new RemoteMethod("msync"); + Set namespacesEligibleForObserverReads = allNamespaces + .stream() + .filter(ns -> rpcClient.isNamespaceObserverReadEligible(ns.getNameserviceId())) + .collect(Collectors.toSet()); + if (namespacesEligibleForObserverReads.isEmpty()) { + asyncCompleteWith(CompletableFuture.completedFuture(null)); + return; + } + rpcClient.invokeConcurrent(namespacesEligibleForObserverReads, method); + } + + @Override + public boolean setReplication(String src, short replication) + throws IOException { + 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); + asyncApply(o -> { + Map results = (Map) o; + return !results.containsValue(false); + }); + } else { + rpcClient.invokeSequential(locations, method, Boolean.class, + Boolean.TRUE); + } + return asyncReturn(boolean.class); + } + + /** + * Checks if the path is a directory and is supposed to be present in all + * subclusters. + * @param src the source path + * @return true if the path is directory and is supposed to be present in all + * subclusters else false in all other scenarios. + * @throws IOException if unable to get the file status. + */ + @Override + public boolean isMultiDestDirectory(String src) throws IOException { + 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; + }); + asyncCompleteWith(completableFuture); + return asyncReturn(Boolean.class); + } + } catch (UnresolvedPathException e) { + LOG.debug("The destination {} is a symlink.", src); + } + asyncCompleteWith(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/async/RouterAsyncNamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncNamenodeProtocol.java new file mode 100644 index 0000000000000..fc461dd22afaf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncNamenodeProtocol.java @@ -0,0 +1,198 @@ +/** + * 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.async; + +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.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RouterNamenodeProtocol; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncApplyFunction; +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 static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; + +/** + * Module that implements all the asynchronous RPC calls in {@link NamenodeProtocol} in the + * {@link RouterRpcServer}. + */ +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(); + } + + /** + * Asynchronously get a list of blocks belonging to datanode + * whose total size equals size. + * + * @see org.apache.hadoop.hdfs.server.balancer.Balancer + * @param datanode a data node + * @param size requested size + * @param minBlockSize each block should be of this minimum Block Size + * @param hotBlockTimeInterval prefer to get blocks which are belong to + * the cold files accessed before the time interval + * @param storageType the given storage type {@link StorageType} + * @return BlocksWithLocations a list of blocks & their locations + * @throws IOException if size is less than or equal to 0 or + datanode does not exist + */ + @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); + asyncApply((AsyncApplyFunction, Object>) map -> { + String nsId = null; + 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; + } + } + // Forward to the proper namenode + if (nsId != null) { + 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); + } else { + asyncComplete(null); + } + }); + return asyncReturn(BlocksWithLocations.class); + } + + /** + * Asynchronously get the current block keys. + * + * @return ExportedBlockKeys containing current block keys + * @throws IOException if there is no namespace available or other ioExceptions. + */ + @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); + } + + /** + * Asynchronously get the most recent transaction ID. + * + * @return The most recent transaction ID that has been synced to + * persistent storage, or applied from persistent storage in the + * case of a non-active node. + * @throws IOException if there is no namespace available or other ioExceptions. + */ + @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); + } + + /** + * Asynchronously get the transaction ID of the most recent checkpoint. + * + * @return The transaction ID of the most recent checkpoint. + * @throws IOException if there is no namespace available or other ioExceptions. + */ + @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); + } + + /** + * Asynchronously get the transaction ID of the most recent checkpoint + * for the given NameNodeFile. + * + * @return The transaction ID of the most recent checkpoint + * for the given NameNodeFile. + * @throws IOException if there is no namespace available or other ioExceptions. + */ + @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); + } + + /** + * Asynchronously request name-node version and storage information. + * + * @return {@link NamespaceInfo} identifying versions and storage information + * of the name-node. + * @throws IOException if there is no namespace available or other ioExceptions. + */ + @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/async/RouterAsyncRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncRpcClient.java new file mode 100644 index 0000000000000..c214adf1f2abb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncRpcClient.java @@ -0,0 +1,630 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.NameNodeProxiesClient; +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.hdfs.server.federation.router.ConnectionContext; +import org.apache.hadoop.hdfs.server.federation.router.RemoteLocationContext; +import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RemoteResult; +import org.apache.hadoop.hdfs.server.federation.router.Router; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcMonitor; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.RouterStateIdContext; +import org.apache.hadoop.hdfs.server.federation.router.ThreadLocalContext; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncApplyFunction; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncCatchFunction; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.StandbyException; +import org.apache.hadoop.security.UserGroupInformation; +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.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; + +import static org.apache.hadoop.hdfs.server.federation.fairness.RouterRpcFairnessConstants.CONCURRENT_NS; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApplyUseExecutor; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCatch; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCompleteWith; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncFinally; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncForEach; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncThrowException; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncTry; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.getCompletableFuture; + +/** + * The {@code RouterAsyncRpcClient} class extends the functionality of the base + * {@code RouterRpcClient} class to provide asynchronous remote procedure call (RPC) + * capabilities for communication with the Hadoop Distributed File System (HDFS) + * NameNodes in a federated environment. + * + *

This class is responsible for managing the asynchronous execution of RPCs to + * multiple NameNodes, which can improve performance and scalability in large HDFS + * deployments. + * + *

The class also includes methods for handling failover scenarios, where it can + * automatically retry operations on alternative NameNodes if the primary NameNode is + * unavailable or in standby mode. + * + * @see RouterRpcClient + */ +public class RouterAsyncRpcClient extends RouterRpcClient{ + private static final Logger LOG = + LoggerFactory.getLogger(RouterAsyncRpcClient.class); + /** Router using this RPC client. */ + private final Router router; + /** Interface to identify the active NN for a nameservice or blockpool ID. */ + private final ActiveNamenodeResolver namenodeResolver; + /** Optional perf monitor. */ + private final RouterRpcMonitor rpcMonitor; + + /** + * Create a router async 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); + this.router = router; + this.namenodeResolver = resolver; + this.rpcMonitor = monitor; + } + + /** + * Invoke method in all locations and return success if any succeeds. + * + * @param The type of the remote location. + * @param locations List of remote locations to call concurrently. + * @param method The remote method and parameters to invoke. + * @return If the call succeeds in any location. + * @throws IOException If any of the calls return an exception. + */ + @Override + public boolean invokeAll( + final Collection locations, final RemoteMethod method) + throws IOException { + invokeConcurrent(locations, method, false, false, + Boolean.class); + asyncApply((ApplyFunction, Object>) + results -> results.containsValue(true)); + return asyncReturn(boolean.class); + } + + /** + * Invokes a method against the ClientProtocol proxy server. If a standby + * exception is generated by the call to the client, retries using the + * alternate server. + *

+ * Re-throws exceptions generated by the remote RPC call as either + * RemoteException or IOException. + * + * @param ugi User group information. + * @param namenodes A prioritized list of namenodes within the same + * nameservice. + * @param useObserver Whether to use observer namenodes. + * @param protocol the protocol of the connection. + * @param method Remote ClientProtocol method to invoke. + * @param params Variable list of parameters matching the method. + * @return The result of invoking the method. + * @throws ConnectException If it cannot connect to any Namenode. + * @throws StandbyException If all Namenodes are in Standby. + * @throws IOException If it cannot invoke the method. + */ + @Override + public Object invokeMethod( + UserGroupInformation ugi, + List namenodes, + boolean useObserver, Class protocol, + Method method, 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()); + } + String nsid = namenodes.get(0).getNameserviceId(); + // transfer threadLocalContext to worker threads of executor. + ThreadLocalContext threadLocalContext = new ThreadLocalContext(); + asyncComplete(null); + asyncApplyUseExecutor((AsyncApplyFunction) o -> { + if (LOG.isDebugEnabled()) { + LOG.debug("Async invoke method : {}, {}, {}, {}", method.getName(), useObserver, + namenodes.toString(), params); + } + threadLocalContext.transfer(); + invokeMethodAsync(ugi, (List) namenodes, + useObserver, protocol, method, params); + }, router.getRpcServer().getAsyncRouterHandlerExecutors().getOrDefault(nsid, + router.getRpcServer().getRouterAsyncHandlerDefaultExecutor())); + return null; + } + + /** + * Asynchronously invokes a method on the specified NameNodes for a given user and operation. + * This method is responsible for the actual execution of the remote method call on the + * NameNodes in a non-blocking manner, allowing for concurrent processing. + * + *

In case of exceptions, the method includes logic to handle retries, failover to standby + * NameNodes, and proper exception handling to ensure that the calling code can respond + * appropriately to different error conditions. + * + * @param ugi The user information under which the method is to be invoked. + * @param namenodes The list of NameNode contexts on which the method will be invoked. + * @param useObserver Whether to use an observer node for the invocation if available. + * @param protocol The protocol class defining the method to be invoked. + * @param method The method to be invoked on the NameNodes. + * @param params The parameters for the method invocation. + */ + private void invokeMethodAsync( + final UserGroupInformation ugi, + final List namenodes, + boolean useObserver, + final Class protocol, final Method method, final Object... params) { + + addClientInfoToCallerContext(ugi); + if (rpcMonitor != null) { + rpcMonitor.proxyOp(); + } + final ExecutionStatus status = new ExecutionStatus(false, useObserver); + Map ioes = new LinkedHashMap<>(); + final ConnectionContext[] connection = new ConnectionContext[1]; + asyncForEach(namenodes.iterator(), + (foreach, namenode) -> { + if (!status.isShouldUseObserver() + && (namenode.getState() == FederationNamenodeServiceState.OBSERVER)) { + asyncComplete(null); + return; + } + String nsId = namenode.getNameserviceId(); + String rpcAddress = namenode.getRpcAddress(); + asyncTry(() -> { + connection[0] = getConnection(ugi, nsId, rpcAddress, protocol); + NameNodeProxiesClient.ProxyAndInfo client = connection[0].getClient(); + invoke(namenode, status.isShouldUseObserver(), 0, method, + client.getProxy(), params); + asyncApply(res -> { + status.setComplete(true); + postProcessResult(method, status, namenode, nsId, client); + foreach.breakNow(); + return res; + }); + }); + asyncCatch((res, ioe) -> { + ioes.put(namenode, ioe); + handleInvokeMethodIOException(namenode, ioe, status, useObserver); + return res; + }, IOException.class); + asyncFinally(res -> { + if (connection[0] != null) { + connection[0].release(); + } + return res; + }); + }); + + asyncApply(res -> { + if (status.isComplete()) { + return res; + } + return handlerAllNamenodeFail(namenodes, method, ioes, params); + }); + } + + /** + * Asynchronously invokes a method on a specified NameNode in the context of the given + * namespace and NameNode information. This method is designed to handle the invocation + * in a non-blocking manner, allowing for improved performance and scalability when + * interacting with the NameNode. + * + * @param namenode The context information for the NameNode. + * @param listObserverFirst Whether to list the observer node first in the invocation list. + * @param retryCount The current retry count for the operation. + * @param method The method to be invoked on the NameNode. + * @param obj The proxy object through which the method will be invoked. + * @param params The parameters for the method invocation. + */ + protected Object invoke( + FederationNamenodeContext namenode, Boolean listObserverFirst, + int retryCount, final Method method, + final Object obj, final Object... params) throws IOException { + try { + Client.setAsynchronousMode(true); + method.invoke(obj, params); + Client.setAsynchronousMode(false); + asyncCatch((AsyncCatchFunction) (o, e) -> { + handlerInvokeException(namenode, listObserverFirst, + retryCount, method, obj, e, params); + }, Throwable.class); + } catch (InvocationTargetException e) { + asyncThrowException(e.getCause()); + } catch (IllegalAccessException | IllegalArgumentException e) { + LOG.error("Unexpected exception while proxying API", e); + asyncThrowException(e); + } + return null; + } + + /** + * Invokes sequential proxy calls to different locations. Continues to invoke + * calls until the success condition is met, or until all locations have been + * attempted. + * + * The success condition may be specified by: + *

    + *
  • An expected result class + *
  • An expected result value + *
+ * + * If no expected result class/values are specified, the success condition is + * a call that does not throw a remote exception. + * + * @param The type of the remote method return. + * @param locations List of locations/nameservices to call concurrently. + * @param remoteMethod The remote method and parameters to invoke. + * @param expectedResultClass In order to be considered a positive result, the + * return type must be of this class. + * @param expectedResultValue In order to be considered a positive result, the + * return value must equal the value of this object. + * @return The result of the first successful call, or if no calls are + * successful, the result of the first RPC call executed. + * @throws IOException if the success condition is not met, return the first + * remote exception generated. + */ + @Override + public T invokeSequential( + final List locations, + final RemoteMethod remoteMethod, Class expectedResultClass, + Object expectedResultValue) throws IOException { + invokeSequential(remoteMethod, locations, expectedResultClass, expectedResultValue); + asyncApply((ApplyFunction) RemoteResult::getResult); + return asyncReturn(expectedResultClass); + } + + /** + * Invokes sequential proxy calls to different locations. Continues to invoke + * calls until the success condition is met, or until all locations have been + * attempted. + * + * The success condition may be specified by: + *
    + *
  • An expected result class + *
  • An expected result value + *
+ * + * If no expected result class/values are specified, the success condition is + * a call that does not throw a remote exception. + * + * This returns RemoteResult, which contains the invoked location as well + * as the result. + * + * @param The type of the remote location. + * @param The type of the remote method return. + * @param remoteMethod The remote method and parameters to invoke. + * @param locations List of locations/nameservices to call concurrently. + * @param expectedResultClass In order to be considered a positive result, the + * return type must be of this class. + * @param expectedResultValue In order to be considered a positive result, the + * return value must equal the value of this object. + * @return The result of the first successful call, or if no calls are + * successful, the result of the first RPC call executed, along with + * the invoked location in form of RemoteResult. + * @throws IOException if the success condition is not met, return the first + * remote exception generated. + */ + @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<>(); + final Object[] firstResult = {null}; + final ExecutionStatus status = new ExecutionStatus(); + Iterator locationIterator = + (Iterator) locations.iterator(); + // Invoke in priority order + asyncForEach(locationIterator, + (foreach, loc) -> { + String ns = loc.getNameserviceId(); + boolean isObserverRead = isObserverReadEligible(ns, m); + List namenodes = + getOrderedNamenodes(ns, isObserverRead); + acquirePermit(ns, ugi, remoteMethod, controller); + asyncTry(() -> { + Class proto = remoteMethod.getProtocol(); + Object[] params = remoteMethod.getParams(loc); + invokeMethod(ugi, namenodes, isObserverRead, proto, m, params); + asyncApply(result -> { + // Check if the result is what we expected + if (isExpectedClass(expectedResultClass, result) && + isExpectedValue(expectedResultValue, result)) { + // Valid result, stop here + @SuppressWarnings("unchecked") R location = (R) loc; + @SuppressWarnings("unchecked") T ret = (T) result; + foreach.breakNow(); + status.setComplete(true); + return new RemoteResult<>(location, ret); + } + if (firstResult[0] == null) { + firstResult[0] = result; + } + return null; + }); + }); + asyncCatch((ret, e) -> { + if (e instanceof IOException) { + IOException ioe = (IOException) e; + // 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 ret; + }, Exception.class); + asyncFinally(ret -> { + releasePermit(ns, ugi, remoteMethod, controller); + return ret; + }); + }); + asyncApply(result -> { + if (status.isComplete()) { + return result; + } + 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 ioe; + } + } + // re-throw the first exception thrown for compatibility + throw thrownExceptions.get(0); + } + // Return the first result, whether it is the value or not + @SuppressWarnings("unchecked") T ret = (T) firstResult[0]; + return new RemoteResult<>(locations.get(0), ret); + }); + return asyncReturn(RemoteResult.class); + } + + /** + * Invokes multiple concurrent proxy calls to different clients. Returns an + * array of results. + *

+ * Re-throws exceptions generated by the remote RPC call as either + * RemoteException or IOException. + * + * @param The type of the remote location. + * @param The type of the remote method return. + * @param locations List of remote locations to call concurrently. + * @param method The remote method and parameters to invoke. + * @param requireResponse If true an exception will be thrown if all calls do + * not complete. If false exceptions are ignored and all data results + * successfully received are returned. + * @param standby If the requests should go to the standby namenodes too. + * @param timeOutMs Timeout for each individual call. + * @param clazz Type of the remote return type. + * @return Result of invoking the method per subcluster: nsId to result. + * @throws IOException If requiredResponse=true and any of the calls throw an + * exception. + */ + @Override + public Map invokeConcurrent( + final Collection locations, final RemoteMethod method, + boolean requireResponse, boolean standby, long timeOutMs, Class clazz) + throws IOException { + invokeConcurrent(locations, method, standby, timeOutMs, clazz); + asyncApply((ApplyFunction>, Object>) + results -> postProcessResult(requireResponse, results)); + return asyncReturn(Map.class); + } + + /** + * Invokes multiple concurrent proxy calls to different clients. Returns an + * array of results. + * + * @param The type of the remote location. + * @param The type of the remote method return. + * @param method The remote method and parameters to invoke. + * @param timeOutMs Timeout for each individual call. + * @param controller Fairness manager to control handlers assigned per NS. + * @param orderedLocations List of remote locations to call concurrently. + * @param callables Invoke method for each NameNode. + * @return Result of invoking the method per subcluster (list of results), + * This includes the exception for each remote location. + * @throws IOException If there are errors invoking the method. + */ + @Override + protected List> getRemoteResults( + RemoteMethod method, long timeOutMs, RouterRpcFairnessPolicyController controller, + List orderedLocations, List> callables) throws IOException { + final UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + final Method m = method.getMethod(); + final CompletableFuture[] futures = + new CompletableFuture[callables.size()]; + int i = 0; + for (Callable callable : callables) { + CompletableFuture future = null; + try { + callable.call(); + future = getCompletableFuture(); + } catch (Exception e) { + future = new CompletableFuture<>(); + future.completeExceptionally(warpCompletionException(e)); + } + futures[i++] = future; + } + + asyncCompleteWith(CompletableFuture.allOf(futures) + .handle((unused, throwable) -> { + try { + return processFutures(method, m, orderedLocations, Arrays.asList(futures)); + } catch (InterruptedException e) { + LOG.error("Unexpected error while invoking API: {}", e.getMessage()); + throw warpCompletionException(new IOException( + "Unexpected error while invoking API " + e.getMessage(), e)); + } finally { + releasePermit(CONCURRENT_NS, ugi, method, controller); + } + })); + return asyncReturn(List.class); + } + + /** + * Invokes a ClientProtocol method against the specified namespace. + *

+ * Re-throws exceptions generated by the remote RPC call as either + * RemoteException or IOException. + * + * @param The type of the remote location. + * @param The type of the remote method return. + * @param location RemoteLocation to invoke. + * @param method The remote method and parameters to invoke. + * @return Result of invoking the method per subcluster (list of results), + * This includes the exception for each remote location. + * @throws IOException If there are errors invoking the method. + */ + @Override + public List> invokeSingle( + T location, RemoteMethod method) throws IOException { + final UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + final Method m = method.getMethod(); + String ns = location.getNameserviceId(); + boolean isObserverRead = isObserverReadEligible(ns, m); + final List namenodes = + getOrderedNamenodes(ns, isObserverRead); + RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController(); + acquirePermit(ns, ugi, method, controller); + asyncTry(() -> { + Class proto = method.getProtocol(); + Object[] paramList = method.getParams(location); + invokeMethod(ugi, namenodes, isObserverRead, proto, m, paramList); + asyncApply((ApplyFunction) result -> { + RemoteResult remoteResult = new RemoteResult<>(location, result); + return Collections.singletonList(remoteResult); + }); + }); + asyncCatch((o, ioe) -> { + throw processException(ioe, location); + }, IOException.class); + asyncFinally(o -> { + releasePermit(ns, ugi, method, controller); + return o; + }); + return asyncReturn(List.class); + } + + /** + * Invokes a ClientProtocol method against the specified namespace. + *

+ * Re-throws exceptions generated by the remote RPC call as either + * RemoteException or IOException. + * + * @param nsId Target namespace for the method. + * @param method The remote method and parameters to invoke. + * @return The result of invoking the method. + * @throws IOException If the invoke generated an error. + */ + @Override + public Object invokeSingle(final String nsId, RemoteMethod method) + throws IOException { + UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + RouterRpcFairnessPolicyController controller = getRouterRpcFairnessPolicyController(); + acquirePermit(nsId, ugi, method, controller); + asyncTry(() -> { + boolean isObserverRead = isObserverReadEligible(nsId, method.getMethod()); + List nns = getOrderedNamenodes(nsId, isObserverRead); + RemoteLocationContext loc = new RemoteLocation(nsId, "/", "/"); + Class proto = method.getProtocol(); + Method m = method.getMethod(); + Object[] params = method.getParams(loc); + invokeMethod(ugi, nns, isObserverRead, proto, m, params); + }); + asyncFinally(o -> { + releasePermit(nsId, ugi, method, controller); + return o; + }); + return null; + } + + /** + * Invokes a single proxy call for a single location. + *

+ * Re-throws exceptions generated by the remote RPC call as either + * RemoteException or IOException. + * + * @param location RemoteLocation to invoke. + * @param remoteMethod The remote method and parameters to invoke. + * @param clazz Class for the return type. + * @param The type of the remote method return. + * @return The result of invoking the method if successful. + * @throws IOException If the invoke generated an error. + */ + public T invokeSingle( + final RemoteLocationContext location, + RemoteMethod remoteMethod, Class clazz) throws IOException { + List locations = Collections.singletonList(location); + invokeSequential(locations, remoteMethod); + return asyncReturn(clazz); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncSnapshot.java new file mode 100644 index 0000000000000..c38d243aa38e5 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncSnapshot.java @@ -0,0 +1,264 @@ +/** + * 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.async; + +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.protocol.ClientProtocol; +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.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RemoteParam; +import org.apache.hadoop.hdfs.server.federation.router.RemoteResult; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.RouterSnapshot; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.ApplyFunction; +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.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; + +/** + * Module that implements all the asynchronous RPC calls related to snapshots in + * {@link ClientProtocol} in the {@link RouterRpcServer}. + */ +public class RouterAsyncSnapshot extends RouterSnapshot { + /** RPC server to receive client calls. */ + private final RouterRpcServer rpcServer; + /** RPC clients to connect to the Namenodes. */ + private final RouterRpcClient rpcClient; + /** Find generic locations. */ + private final ActiveNamenodeResolver namenodeResolver; + + public RouterAsyncSnapshot(RouterRpcServer server) { + super(server); + this.rpcServer = server; + this.rpcClient = this.rpcServer.getRPCClient(); + this.namenodeResolver = rpcServer.getNamenodeResolver(); + } + + /** + * Asynchronously creates a snapshot with the given root and name. + * This method checks the operation category and then invokes the createSnapshot + * method concurrently across all namespaces, returning the first successful response. + * + * @param snapshotRoot The root path of the snapshot. + * @param snapshotName The name of the snapshot. + * @return The path of the created snapshot. + * @throws IOException If an I/O error occurs. + */ + @Override + public String createSnapshot(String snapshotRoot, String snapshotName) throws IOException { + 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)) { + rpcClient.invokeConcurrent(locations, method, String.class); + asyncApply((ApplyFunction, String>) + results -> { + Map.Entry firstelement = + results.entrySet().iterator().next(); + RemoteLocation loc = firstelement.getKey(); + String result = firstelement.getValue(); + return result.replaceFirst(loc.getDest(), loc.getSrc()); + }); + } else { + rpcClient.invokeSequential(method, locations, String.class, null); + asyncApply((ApplyFunction, String>) + response -> { + RemoteLocation loc = response.getLocation(); + String invokedResult = response.getResult(); + return invokedResult.replaceFirst(loc.getDest(), loc.getSrc()); + }); + } + return asyncReturn(String.class); + } + + /** + * Asynchronously get an array of snapshottable directory listings. + * This method checks the operation category and then invokes the + * getSnapshottableDirListing method concurrently across all namespaces, merging + * the results into a single array. + * + * @return Array of SnapshottableDirectoryStatus. + * @throws IOException If an I/O error occurs. + */ + @Override + public SnapshottableDirectoryStatus[] getSnapshottableDirListing() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getSnapshottableDirListing"); + Set nss = namenodeResolver.getNamespaces(); + rpcClient.invokeConcurrent( + nss, method, true, false, SnapshottableDirectoryStatus[].class); + asyncApply((ApplyFunction, + SnapshottableDirectoryStatus[]>) + ret -> RouterRpcServer.merge(ret, SnapshottableDirectoryStatus.class)); + return asyncReturn(SnapshottableDirectoryStatus[].class); + } + + /** + * Asynchronously get an array of snapshot listings for the given snapshot root. + * This method checks the operation category and then invokes the + * getSnapshotListing method, either sequentially or concurrently based on the + * configuration, and returns the merged results. + * + * @param snapshotRoot The root path of the snapshots. + * @return Array of SnapshotStatus. + * @throws IOException If an I/O error occurs. + */ + @Override + public SnapshotStatus[] getSnapshotListing(String snapshotRoot) throws IOException { + 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)) { + rpcClient.invokeConcurrent( + locations, remoteMethod, true, false, SnapshotStatus[].class); + asyncApply((ApplyFunction, SnapshotStatus[]>) + 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; + }); + } else { + rpcClient + .invokeSequential(remoteMethod, locations, SnapshotStatus[].class, + null); + asyncApply((ApplyFunction, SnapshotStatus[]>) + invokedResponse -> { + RemoteLocation loc = invokedResponse.getLocation(); + SnapshotStatus[] response = invokedResponse.getResult(); + for (SnapshotStatus s : response) { + String mountPath = DFSUtil.bytes2String(s.getParentFullPath()). + replaceFirst(loc.getDest(), loc.getSrc()); + s.setParentFullPath(DFSUtil.string2Bytes(mountPath)); + } + return response; + }); + } + return asyncReturn(SnapshotStatus[].class); + } + + /** + * Asynchronously get a snapshot diff report for the given root and snapshot names. + * This method checks the operation category and then invokes the + * getSnapshotDiffReport method, either sequentially or concurrently based on the + * configuration, and returns the result. + * + * @param snapshotRoot The root path of the snapshot. + * @param earlierSnapshotName The name of the earlier snapshot. + * @param laterSnapshotName The name of the later snapshot. + * @return SnapshotDiffReport for the snapshots. + * @throws IOException If an I/O error occurs. + */ + @Override + public SnapshotDiffReport getSnapshotDiffReport( + String snapshotRoot, String earlierSnapshotName, + String laterSnapshotName) throws IOException { + 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)) { + rpcClient.invokeConcurrent( + locations, remoteMethod, true, false, SnapshotDiffReport.class); + asyncApply((ApplyFunction, SnapshotDiffReport>) + ret -> ret.values().iterator().next()); + return asyncReturn(SnapshotDiffReport.class); + } else { + return rpcClient.invokeSequential( + locations, remoteMethod, SnapshotDiffReport.class, null); + } + } + + /** + * Asynchronously get a snapshot diff report listing for the given root and snapshot names. + * This method checks the operation category and then invokes the + * getSnapshotDiffReportListing method, either sequentially or concurrently based + * on the configuration, and returns the result. + * + * @param snapshotRoot The root path of the snapshot. + * @param earlierSnapshotName The name of the earlier snapshot. + * @param laterSnapshotName The name of the later snapshot. + * @param startPath The starting path for the diff report. + * @param index The index for the diff report listing. + * @return SnapshotDiffReportListing for the snapshots. + * @throws IOException If an I/O error occurs. + */ + @Override + public SnapshotDiffReportListing getSnapshotDiffReportListing( + String snapshotRoot, String earlierSnapshotName, String laterSnapshotName, + byte[] startPath, int index) throws IOException { + 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)) { + rpcClient.invokeConcurrent(locations, remoteMethod, false, false, + SnapshotDiffReportListing.class); + asyncApply((ApplyFunction, + SnapshotDiffReportListing>) ret -> { + Collection listings = ret.values(); + SnapshotDiffReportListing listing0 = listings.iterator().next(); + return listing0; + }); + return asyncReturn(SnapshotDiffReportListing.class); + } else { + return rpcClient.invokeSequential( + locations, remoteMethod, SnapshotDiffReportListing.class, null); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncStoragePolicy.java new file mode 100644 index 0000000000000..cf23d0f7cc832 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncStoragePolicy.java @@ -0,0 +1,90 @@ +/** + * 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.async; + +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RemoteParam; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.RouterStoragePolicy; +import org.apache.hadoop.hdfs.server.namenode.NameNode; + +import java.io.IOException; +import java.util.List; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; + +/** + * Module that implements all the asynchronous RPC calls in + * {@link org.apache.hadoop.hdfs.protocol.ClientProtocol} related to + * Storage Policy in the {@link RouterRpcServer}. + */ +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(); + } + + /** + * Asynchronously get the storage policy for a given path. + * This method checks the operation category and then invokes the + * getStoragePolicy method sequentially for the given path. + * + * @param path The path for which to retrieve the storage policy. + * @return The BlockStoragePolicy for the given path. + * @throws IOException If an I/O error occurs. + */ + @Override + public BlockStoragePolicy getStoragePolicy(String path) + throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ, true); + + List locations = + rpcServer.getLocationsForPath(path, false, false); + RemoteMethod method = new RemoteMethod("getStoragePolicy", + new Class[] {String.class}, + new RemoteParam()); + rpcClient.invokeSequential(locations, method); + return asyncReturn(BlockStoragePolicy.class); + } + + /** + * Asynchronously get an array of all available storage policies. + * This method checks the operation category and then invokes the + * getStoragePolicies method across all available namespaces. + * + * @return An array of BlockStoragePolicy. + * @throws IOException If an I/O error occurs. + */ + @Override + public BlockStoragePolicy[] getStoragePolicies() throws IOException { + rpcServer.checkOperation(NameNode.OperationCategory.READ); + + RemoteMethod method = new RemoteMethod("getStoragePolicies"); + rpcServer.invokeAtAvailableNsAsync(method, BlockStoragePolicy[].class); + return asyncReturn(BlockStoragePolicy[].class); + } +} \ 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/async/RouterAsyncUserProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncUserProtocol.java new file mode 100644 index 0000000000000..68b5aa8528434 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncUserProtocol.java @@ -0,0 +1,132 @@ +/** + * 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.async; + +import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcClient; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.federation.router.RouterUserProtocol; + +import java.io.IOException; +import java.util.Map; +import java.util.Set; +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.router.async.utils.ApplyFunction; +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 static org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer.merge; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; + +/** + * Module that implements all the asynchronous RPC calls in + * {@link RefreshUserMappingsProtocol} {@link GetUserMappingsProtocol} in the + * {@link RouterRpcServer}. + */ +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(); + } + + /** + * Asynchronously refresh user to group mappings. + * + * @throws IOException raised on errors performing I/O. + */ + @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(); + asyncComplete(null); + } else { + RemoteMethod method = new RemoteMethod(RefreshUserMappingsProtocol.class, + "refreshUserToGroupsMappings"); + rpcClient.invokeConcurrent(nss, method); + } + } + + /** + * Asynchronously refresh superuser proxy group list. + * + * @throws IOException raised on errors performing I/O. + */ + @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(); + asyncComplete(null); + } else { + RemoteMethod method = new RemoteMethod(RefreshUserMappingsProtocol.class, + "refreshSuperUserGroupsConfiguration"); + rpcClient.invokeConcurrent(nss, method); + } + } + + /** + * Asynchronously get the groups which are mapped to the given user. + * + * @param user The user to get the groups for. + * @return The set of groups the user belongs to. + * @throws IOException raised on errors performing I/O. + */ + @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()) { + asyncComplete(UserGroupInformation.createRemoteUser(user) + .getGroupNames()); + } else { + RemoteMethod method = new RemoteMethod(GetUserMappingsProtocol.class, + "getGroupsForUser", new Class[] {String.class}, user); + rpcClient.invokeConcurrent(nss, method, String[].class); + asyncApply((ApplyFunction, String[]>) + results -> merge(results, String.class)); + } + return asyncReturn(String[].class); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/package-info.java new file mode 100644 index 0000000000000..e0ecc4b36f64c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/package-info.java @@ -0,0 +1,31 @@ +/** + * 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. + */ + +/** + * This package contains classes that facilitate asynchronous operations within the Hadoop + * Distributed File System (HDFS) Federation router. These classes are designed to work with + * the Hadoop ecosystem, providing utilities and interfaces to perform non-blocking tasks that + * can improve the performance and responsiveness of HDFS operations. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving + +package org.apache.hadoop.hdfs.server.federation.router.async; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/ApplyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/ApplyFunction.java new file mode 100644 index 0000000000000..721adf1e4d632 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/ApplyFunction.java @@ -0,0 +1,89 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; + +/** + * Represents a function that accepts a value of type T and produces a result of type R. + * This interface extends {@link Async} and provides methods to apply the function + * asynchronously using {@link CompletableFuture}. + * + *

ApplyFunction is used to implement the following semantics:

+ *
+ * {@code
+ *    T res = doAsync(input);
+ *    // Can use ApplyFunction
+ *    R result = thenApply(res);
+ * }
+ * 
+ * + * @param the type of the input to the function + * @param the type of the result of the function + */ +@FunctionalInterface +public interface ApplyFunction extends Async{ + + /** + * Applies this function to the given argument. + * + * @param t the function argument + * @return the function result + * @throws IOException if an I/O error occurs + */ + R apply(T t) throws IOException; + + /** + * Applies this function asynchronously to the result of the given {@link CompletableFuture}. + * The function is executed on the same thread as the completion of the given future. + * + * @param in the input future + * @return a new future that holds the result of the function application + */ + default CompletableFuture apply(CompletableFuture in) { + return in.thenApply(t -> { + try { + return ApplyFunction.this.apply(t); + } catch (IOException e) { + throw warpCompletionException(e); + } + }); + } + + /** + * Applies this function asynchronously to the result of the given {@link CompletableFuture}, + * using the specified executor for the asynchronous computation. + * + * @param in the input future + * @param executor the executor to use for the asynchronous computation + * @return a new future that holds the result of the function application + */ + default CompletableFuture apply(CompletableFuture in, Executor executor) { + return in.thenApplyAsync(t -> { + try { + return ApplyFunction.this.apply(t); + } catch (IOException e) { + throw warpCompletionException(e); + } + }, executor); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/Async.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/Async.java new file mode 100644 index 0000000000000..09cd649b7e61c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/Async.java @@ -0,0 +1,115 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.CompletionException; + +/** + * An interface for asynchronous operations, providing utility methods + * and constants related to asynchronous computations. + * + * @param The type of the result of the asynchronous operation + */ +public interface Async { + + /** + * A thread-local variable to store the {@link CompletableFuture} instance for the current thread. + *

+ * Note: After executing an asynchronous method, the thread stores the CompletableFuture + * of the asynchronous method in the thread's local variable + */ + ThreadLocal> CUR_COMPLETABLE_FUTURE + = new ThreadLocal<>(); + + /** + * Sets the {@link CompletableFuture} instance for the current thread. + * + * @param completableFuture The {@link CompletableFuture} instance to be set + * @param The type of the result in the CompletableFuture + */ + default void setCurCompletableFuture(CompletableFuture completableFuture) { + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) completableFuture); + } + + /** + * Gets the {@link CompletableFuture} instance for the current thread. + * + * @return The {@link CompletableFuture} instance for the current thread, + * or {@code null} if not set + */ + default CompletableFuture getCurCompletableFuture() { + return (CompletableFuture) CUR_COMPLETABLE_FUTURE.get(); + } + + /** + * Blocks and retrieves the result of the {@link CompletableFuture} instance + * for the current thread. + * + * @return The result of the CompletableFuture, or {@code null} if the thread was interrupted + * @throws IOException If the completion exception to the CompletableFuture + * is an IOException or a subclass of it + */ + default R result() throws IOException { + try { + CompletableFuture completableFuture = + (CompletableFuture) CUR_COMPLETABLE_FUTURE.get(); + assert completableFuture != null; + return completableFuture.get(); + } catch (InterruptedException e) { + return null; + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof IOException) { + throw (IOException)cause; + } + throw new IOException(e); + } + } + + /** + * Extracts the real cause of an exception wrapped by CompletionException. + * + * @param e The incoming exception, which may be a CompletionException. + * @return Returns the real cause of the original exception, + * or the original exception if there is no cause. + */ + static Throwable unWarpCompletionException(Throwable e) { + if (e instanceof CompletionException) { + if (e.getCause() != null) { + return e.getCause(); + } + } + return e; + } + + /** + * Wraps the incoming exception in a new CompletionException. + * + * @param e The incoming exception, which may be any type of Throwable. + * @return Returns a new CompletionException with the original exception as its cause. + */ + static CompletionException warpCompletionException(Throwable e) { + if (e instanceof CompletionException) { + return (CompletionException) e; + } + return new CompletionException(e); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncApplyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncApplyFunction.java new file mode 100644 index 0000000000000..0355efabbc98b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncApplyFunction.java @@ -0,0 +1,162 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; + +/** + * The AsyncApplyFunction interface represents a function that + * asynchronously accepts a value of type T and produces a result + * of type R. This interface extends {@link ApplyFunction} and is + * designed to be used with asynchronous computation frameworks, + * such as Java's {@link java.util.concurrent.CompletableFuture}. + * + *

An implementation of this interface is expected to perform an + * asynchronous operation and return a result, which is typically + * represented as a {@code CompletableFuture}. This allows for + * non-blocking execution of tasks and is particularly useful for + * I/O operations or any operation that may take a significant amount + * of time to complete.

+ * + *

AsyncApplyFunction is used to implement the following semantics:

+ *
+ * {@code
+ *    T res = doAsync1(input);
+ *    // Can use AsyncApplyFunction
+ *    R result = doAsync2(res);
+ * }
+ * 
+ * + * @param the type of the input to the function + * @param the type of the result of the function + * @see ApplyFunction + * @see java.util.concurrent.CompletableFuture + */ +@FunctionalInterface +public interface AsyncApplyFunction extends ApplyFunction { + + /** + * Asynchronously applies this function to the given argument. + * + *

This method is intended to initiate the function application + * without waiting for the result. It is typically used when the + * result of the operation is not required immediately or when the + * operation is part of a larger asynchronous workflow.

+ * + * @param t the function argument + * @throws IOException if an I/O error occurs during the application + * of the function + */ + void applyAsync(T t) throws IOException; + + /** + * Synchronously applies this function to the given argument and + * returns the result. + * + *

This method waits for the asynchronous operation to complete + * and returns its result. It is useful when the result is needed + * immediately and the calling code cannot proceed without it.

+ * + * @param t the function argument + * @return the result of applying the function to the argument + * @throws IOException if an I/O error occurs during the application + * of the function + */ + @Override + default R apply(T t) throws IOException { + applyAsync(t); + return result(); + } + + /** + * Initiates the asynchronous application of this function to the given result. + *

+ * This method calls applyAsync to start the asynchronous operation and then retrieves + * the current thread's CompletableFuture using getCurCompletableFuture. + * It returns this CompletableFuture, which will be completed with the result of the + * asynchronous operation once it is finished. + *

+ * This method is useful for chaining with other asynchronous operations, as it allows the + * current operation to be part of a larger asynchronous workflow. + * + * @param t the function argument + * @return a CompletableFuture that will be completed with the result of the + * asynchronous operation + * @throws IOException if an I/O error occurs during the initiation of the asynchronous operation + */ + default CompletableFuture async(T t) throws IOException { + applyAsync(t); + CompletableFuture completableFuture = getCurCompletableFuture(); + assert completableFuture != null; + return completableFuture; + } + + /** + * Asynchronously applies this function to the result of the given + * CompletableFuture. + * + *

This method chains the function application to the completion + * of the input future. It returns a new CompletableFuture that + * completes with the function's result when the input future + * completes.

+ * + * @param in the input future + * @return a new CompletableFuture that holds the result of the + * function application + */ + @Override + default CompletableFuture apply(CompletableFuture in) { + return in.thenCompose(t -> { + try { + return async(t); + } catch (IOException e) { + throw warpCompletionException(e); + } + }); + } + + /** + * Asynchronously applies this function to the result of the given + * CompletableFuture, using the specified executor for the + * asynchronous computation. + * + *

This method allows for more control over the execution + * context of the asynchronous operation, such as running the + * operation in a separate thread or thread pool.

+ * + * @param in the input future + * @param executor the executor to use for the asynchronous + * computation + * @return a new CompletableFuture that holds the result of the + * function application + */ + @Override + default CompletableFuture apply(CompletableFuture in, Executor executor) { + return in.thenComposeAsync(t -> { + try { + return async(t); + } catch (IOException e) { + throw warpCompletionException(e); + } + }, executor); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncBiFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncBiFunction.java new file mode 100644 index 0000000000000..8e7ac35726bfa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncBiFunction.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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +/** + * The {@code AsyncBiFunction} interface represents a bi-function that + * asynchronously accepts two arguments and produces a result. This interface + * extends the {@link Async} interface and provides a method to apply the + * function asynchronously. + * + *

Implementations of this interface are expected to perform an + * asynchronous operation that takes two parameters of types T and P, and + * returns a result of type R. The asynchronous operation is typically + * represented as a {@link CompletableFuture} of the result type R.

+ * + *

For example, an implementation of this interface might perform an + * asynchronous computation using the two input parameters and return a + * future representing the result of that computation.

+ * + * @param the type of the first argument to the function + * @param

the type of the second argument to the function + * @param the type of the result of the function + * @see Async + */ +@FunctionalInterface +public interface AsyncBiFunction extends Async{ + + /** + * Asynchronously applies this function to the given arguments. + * + *

This method is intended to initiate the function application + * without waiting for the result. It should be used when the + * operation can be performed in the background, and the result + * is not required immediately.

+ * + * @param t the first argument to the function + * @param p the second argument to the function + * @throws IOException if an I/O error occurs during the application of the function + */ + void applyAsync(T t, P p) throws IOException; + + /** + * Initiates the asynchronous application of this function to the given result. + *

+ * This method calls applyAsync to start the asynchronous operation and then retrieves + * the current thread's CompletableFuture using getCurCompletableFuture. + * It returns this CompletableFuture, which will be completed with the result of the + * asynchronous operation once it is finished. + *

+ * This method is useful for chaining with other asynchronous operations, as it allows the + * current operation to be part of a larger asynchronous workflow. + * + * @param t the first argument to the function + * @param p the second argument to the function + * @return a CompletableFuture that will be completed with the result of the + * asynchronous operation + * @throws IOException if an I/O error occurs during the initiation of the asynchronous operation + */ + default CompletableFuture async(T t, P p) throws IOException { + applyAsync(t, p); + CompletableFuture completableFuture = getCurCompletableFuture(); + assert completableFuture != null; + return completableFuture; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncCatchFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncCatchFunction.java new file mode 100644 index 0000000000000..714f57827ffe0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncCatchFunction.java @@ -0,0 +1,174 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.unWarpCompletionException; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; + +/** + * The AsyncCatchFunction interface represents a function that handles exceptions + * occurring within an asynchronous operation. It extends the CatchFunction + * interface and adds the capability to perform asynchronous exception handling. + * + *

This interface is part of the asynchronous utilities provided by the Hadoop + * Distributed File System (HDFS) Federation router. It is used in conjunction + * with other asynchronous interfaces such as AsyncRun to build complex, + * non-blocking operations.

+ * + *

An implementation of this interface should define how to handle a caught + * exception asynchronously. It takes two parameters: the result of the + * asynchronous operation (if any) and the caught exception. The function can then + * initiate an asynchronous process to handle the exception, which may involve + * logging the error, performing a recovery operation, or any other custom logic.

+ * + *

For example, the applyAsync method is intended to be called when an exception + * is caught during an asynchronous operation. It should initiate the asynchronous + * process to handle the exception without blocking the main execution thread.

+ * + *

The default method apply is provided to allow synchronous operation in case + * the asynchronous handling is not required. It simply calls applyAsync and waits + * for the result.

+ * + *

The default method async is provided to allow chaining with other asynchronous + * operations. It calls applyAsync and returns a CompletableFuture that completes + * with the result of the operation.

+ * + *

AsyncCatchFunction is used to implement the following semantics:

+ *
+ * {@code
+ *    try{
+ *      R res = doAsync1(input);
+ *    } catch(E e) {
+ *      // Can use AsyncCatchFunction
+ *      R result = doAsync2(res, e);
+ *    }
+ * }
+ * 
+ * + * @param the type of the result of the asynchronous operation + * @param the type of the exception to catch, extending Throwable + * @see CatchFunction + * @see AsyncRun + */ +@FunctionalInterface +public interface AsyncCatchFunction + extends CatchFunction { + + /** + * Asynchronously applies this function to the given exception. + * + *

This method is intended to be called when an exception is caught + * during an asynchronous operation. It should initiate the asynchronous process + * to handle the exception without blocking the main execution thread. + * The actual handling of the exception, such as logging the error, performing + * a recovery operation, or any other custom logic, should be implemented in this + * method.

+ * + * @param r the result of the asynchronous operation, if any; may be null + * @param e the caught exception + * @throws IOException if an I/O error occurs during the application of the function + */ + void applyAsync(R r, E e) throws IOException; + + /** + * Synchronously applies this function to the given result and exception. + *

+ * This method first calls {@code applyAsync} to initiate the asynchronous handling + * of the exception. Then, it waits for the asynchronous operation to complete + * by calling {@code result}, which retrieves the result of the current + * thread's {@link CompletableFuture}. + *

+ * + * @param r the result of the asynchronous operation, if any; may be null + * @param e the caught exception + * @return the result after applying the function + * @throws IOException if an I/O error occurs during the application of the function + */ + @Override + default R apply(R r, E e) throws IOException { + applyAsync(r, e); + return result(); + } + + /** + * Initiates the asynchronous application of this function to the given result and exception. + *

+ * This method calls applyAsync to start the asynchronous operation and then retrieves + * the current thread's CompletableFuture using getCurCompletableFuture. + * It returns this CompletableFuture, which will be completed with the result of the + * asynchronous operation once it is finished. + *

+ * This method is useful for chaining with other asynchronous operations, as it allows the + * current operation to be part of a larger asynchronous workflow. + * + * @param r the result of the asynchronous operation, if any; may be null + * @param e the caught exception + * @return a CompletableFuture that will be completed with the result of the + * asynchronous operation + * @throws IOException if an I/O error occurs during the initiation of the asynchronous operation + */ + default CompletableFuture async(R r, E e) throws IOException { + applyAsync(r, e); + CompletableFuture completableFuture = getCurCompletableFuture(); + assert completableFuture != null; + return completableFuture; + } + + /** + * Applies the catch function to a {@link CompletableFuture}, handling exceptions of a + * specified type. + *

+ * This method is a default implementation that provides a way to integrate exception + * handling into a chain of asynchronous operations. It takes a {@code CompletableFuture} + * and a class object representing the exception type to catch. The method then completes + * the future with the result of applying this catch function to the input future and the + * specified exception type. + *

+ * If the input future completes exceptionally with an instance of the specified exception + * type, the catch function is applied to the exception. Otherwise, if the future + * completes with a different type of exception or normally, the original result or + * exception is propagated. + * + * @param in the input {@code CompletableFuture} to which the catch function is applied + * @param eClazz the class object representing the exception type to catch + * @return a new {@code CompletableFuture} that completes with the result of applying + * the catch function, or propagates the original exception if it does not match + * the specified type + */ + @Override + default CompletableFuture apply( + CompletableFuture in, Class eClazz) { + return in.handle((r, e) -> { + if (e == null) { + return in; + } + Throwable readException = unWarpCompletionException(e); + if (eClazz.isInstance(readException)) { + try { + return async(r, (E) readException); + } catch (IOException ex) { + throw warpCompletionException(ex); + } + } + throw warpCompletionException(e); + }).thenCompose(result -> result); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncForEachRun.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncForEachRun.java new file mode 100644 index 0000000000000..ae984a84b4627 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncForEachRun.java @@ -0,0 +1,185 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; + +/** + * The AsyncForEachRun class is part of the asynchronous operation utilities + * within the Hadoop Distributed File System (HDFS) Federation router. + * It provides the functionality to perform asynchronous operations on each + * element of an Iterator, applying a given async function. + * + *

This class is designed to work with other asynchronous interfaces and + * utility classes to enable complex asynchronous workflows. It allows for + * non-blocking execution of tasks, which can improve the performance and + * responsiveness of HDFS operations.

+ * + *

The class implements the AsyncRun interface, which means it can be used + * in asynchronous task chains. It maintains an Iterator of elements to + * process, an asyncDoOnce to apply to each element.

+ * + *

The run method initiates the asynchronous operation, and the doOnce + * method recursively applies the asyncDoOnce to each element and handles + * the results. If the shouldBreak flag is set, the operation is completed + * with the current result.

+ * + *

AsyncForEachRun is used to implement the following semantics:

+ *
+ * {@code
+ * for (I element : elements) {
+ *     R result = asyncDoOnce(element);
+ * }
+ * return result;
+ * }
+ * 
+ * + * @param the type of the elements being iterated over + * @param the type of the final result after applying the thenApply function + * @see AsyncRun + * @see AsyncBiFunction + */ +public class AsyncForEachRun implements AsyncRun { + + // Indicates whether the iteration should be broken immediately + // after the next asynchronous operation is completed. + private boolean shouldBreak = false; + // The Iterator over the elements to process asynchronously. + private Iterator iterator; + // The async function to apply to each element from the iterator. + private AsyncBiFunction, I, R> asyncDoOnce; + + /** + * Initiates the asynchronous foreach operation by starting the iteration process + * over the elements provided by the iterator. This method sets up the initial + * call to doOnce(R) with a null result, which begins the recursive + * application of the async function to each element of the iterator. + * + *

This method is an implementation of the {@link AsyncRun} interface's + * {@code run} method, allowing it to be used in a chain of asynchronous + * operations. It is responsible for starting the asynchronous processing and + * handling the completion of the operation through the internal + * {@link CompletableFuture}.

+ * + *

If an exception occurs during the first call to {@code doOnce}, the + * exception is caught and the internal CompletableFuture is completed + * exceptionally with a {@link CompletionException} wrapping the original + * IOException.

+ * + *

After initiating the operation, the method sets the current thread's + * {@link Async} {@link CompletableFuture} by calling + * {@link #setCurCompletableFuture(CompletableFuture)} with the internal result + * CompletableFuture. This allows other parts of the asynchronous workflow to + * chain further operations or handle the final result once the foreach loop + * completes.

+ * + * @see AsyncRun + * @see Async#setCurCompletableFuture(CompletableFuture) + */ + @Override + public void run() { + if (iterator == null || !iterator.hasNext()) { + setCurCompletableFuture(CompletableFuture.completedFuture(null)); + return; + } + CompletableFuture result; + try { + result = doOnce(iterator.next()); + } catch (IOException ioe) { + result = new CompletableFuture<>(); + result.completeExceptionally(warpCompletionException(ioe)); + } + setCurCompletableFuture(result); + } + + /** + * Recursively applies the async function to the next element of the iterator + * and handles the result. This method is called for each iteration of the + * asynchronous foreach loop, applying the async function to each element + * and chaining the results. + * + *

If the iterator has no more elements, the CompletableFuture held by this + * class is completed with the last result. If an exception occurs during + * the application of the async function, it is propagated to the + * CompletableFuture, which completes exceptionally.

+ * + *

This method is designed to be called by the {@link #run()} method and + * handles the iteration logic, including breaking the loop if the + * {@link #shouldBreak} flag is set to true.

+ * + * @param element The current element from the async function application. + * @throws IOException if an I/O error occurs during the application of the async function. + */ + private CompletableFuture doOnce(I element) throws IOException { + CompletableFuture completableFuture = asyncDoOnce.async(AsyncForEachRun.this, element); + return completableFuture.thenCompose(res -> { + if (shouldBreak || !iterator.hasNext()) { + return completableFuture; + } + try { + return doOnce(iterator.next()); + } catch (IOException e) { + throw warpCompletionException(e); + } + }); + } + + /** + * Triggers the termination of the current asynchronous iteration. + * + *

This method is used to break out of the asynchronous for-each loop + * prematurely. It sets a flag that indicates the iteration should be + * terminated at the earliest opportunity. This is particularly useful when + * the processing logic determines that further iteration is unnecessary + * or when a specific condition has been met.

+ * + *

Once this method is called, the next time the loop is about to process + * a new element, it will check the flag and cease operation, allowing the + * application to move on to the next step or complete the task.

+ */ + public void breakNow() { + shouldBreak = true; + } + + /** + * Sets the Iterator for the elements to be processed in the asynchronous operation. + * + * @param forEach The Iterator over the elements. + * @return The current AsyncForEachRun instance for chaining. + */ + public AsyncForEachRun forEach(Iterator forEach) { + this.iterator = forEach; + return this; + } + + /** + * Sets the async function to apply to each element from the iterator. + * + * @param asyncDo The async function. + * @return The current AsyncForEachRun instance for chaining. + */ + public AsyncForEachRun asyncDo(AsyncBiFunction, I, R> asyncDo) { + this.asyncDoOnce = asyncDo; + return this; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncRun.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncRun.java new file mode 100644 index 0000000000000..e619a026e1915 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncRun.java @@ -0,0 +1,74 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +/** + * The AsyncRun interface represents an asynchronous operation that can be + * executed in the context of the Hadoop Distributed File System (HDFS) + * Federation router. Implementations of this interface are responsible for + * performing a task and completing a {@link CompletableFuture} with the + * result of the operation. + * + *

+ * The {@code run} method of this interface is intended to be used in an + * asynchronous workflow, where the operation may involve I/O tasks or + * other long-running processes. By implementing this interface, classes + * can define custom asynchronous behavior that can be chained with other + * asynchronous operations using utility methods provided by the + * {@link AsyncUtil} class.

+ * + *

+ * For example, an implementation of AsyncRun could perform a non-blocking + * read or write operation to HDFS, and upon completion, it could use + * AsyncUtil methods to handle the result or propagate any exceptions that + * occurred during the operation.

+ * + * @param the type of the result produced by the asynchronous operation + * @see AsyncUtil + */ +@FunctionalInterface +public interface AsyncRun extends Async { + + /** + * Executes the asynchronous operation represented by this AsyncRun instance. + * This method is expected to perform the operation and, upon completion, + * complete the current thread's {@link CompletableFuture} with the result. + * + * @throws IOException if an I/O error occurs during the execution of the operation + */ + void run() throws IOException; + + /** + * Provides an asynchronous version of the {@code run} method, which returns a + * {@link CompletableFuture} representing the result of the operation. + * This method is typically used in an asynchronous workflow to initiate the + * operation without waiting for its completion. + * + * @return a CompletableFuture that completes with the result of the operation + * @throws IOException if an I/O error occurs during the initiation of the operation + */ + default CompletableFuture async() throws IOException { + run(); + CompletableFuture completableFuture = getCurCompletableFuture(); + assert completableFuture != null; + return completableFuture; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncUtil.java new file mode 100644 index 0000000000000..7884c229aab78 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncUtil.java @@ -0,0 +1,411 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.function.Function; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.CUR_COMPLETABLE_FUTURE; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; + +/** + * The AsyncUtil class provides a collection of utility methods to simplify + * the implementation of asynchronous operations using Java's CompletableFuture. + * It encapsulates common patterns such as applying functions, handling exceptions, + * and executing tasks in a non-blocking manner. This class is designed to work + * with Hadoop's asynchronous router operations in HDFS Federation. + * + *

The utility methods support a fluent-style API, allowing for the chaining of + * asynchronous operations. For example, after an asynchronous operation completes, + * a function can be applied to its result, and the process can continue with + * the new result. This is particularly useful for complex workflows that require + * multiple steps, where each step depends on the completion of the previous one.

+ * + *

The class also provides methods to handle exceptions that may occur during a + * synchronous operation. This ensures that error handling is integrated smoothly + * into the workflow, allowing for robust and fault-tolerant applications.

+ * + * @see CompletableFuture + * @see ApplyFunction + * @see AsyncApplyFunction + * @see AsyncRun + * @see AsyncForEachRun + * @see CatchFunction + * @see AsyncCatchFunction + * @see FinallyFunction + * @see AsyncBiFunction + */ +public final class AsyncUtil { + private static final Boolean BOOLEAN_RESULT = false; + private static final Long LONG_RESULT = -1L; + private static final Integer INT_RESULT = -1; + private static final Object NULL_RESULT = null; + + private AsyncUtil(){} + + /** + * Provides a default value based on the type specified. + * + * @param clazz The {@link Class} object representing the type of the value + * to be returned. + * @param The type of the value to be returned. + * @return An object with a value determined by the type: + *
    + *
  • {@code false} if {@code clazz} is {@link Boolean}, + *
  • -1 if {@code clazz} is {@link Long}, + *
  • -1 if {@code clazz} is {@link Integer}, + *
  • {@code null} for any other type. + *
+ */ + public static R asyncReturn(Class clazz) { + if (clazz == null) { + return null; + } + if (clazz.equals(Boolean.class) + || clazz.equals(boolean.class)) { + return (R) BOOLEAN_RESULT; + } else if (clazz.equals(Long.class) + || clazz.equals(long.class)) { + return (R) LONG_RESULT; + } else if (clazz.equals(Integer.class) + || clazz.equals(int.class)) { + return (R) INT_RESULT; + } + return (R) NULL_RESULT; + } + + /** + * Synchronously returns the result of the current asynchronous operation. + * This method is designed to be used in scenarios where the result of an + * asynchronous operation is needed synchronously, and it is known that + * the operation has completed. + * + *

The method retrieves the current thread's {@link CompletableFuture} and + * attempts to get the result. If the future is not yet complete, this + * method will block until the result is available. If the future completed + * exceptionally, the cause of the exception is thrown as a runtime + * exception wrapped in an {@link ExecutionException}.

+ * + *

This method is typically used after an asynchronous operation has been + * initiated and the caller needs to obtain the result in a synchronous + * manner, for example, when bridging between asynchronous and synchronous + * code paths.

+ * + * @param the type of the result to be returned + * @param clazz the {@link Class} object representing the type of the value + * to be returned, used to cast the result to the correct type + * @return the result of the asynchronous operation as an object of the + * specified class + * @throws Exception if an error occurs during the synchronous retrieval of + * the result, including the original exception if the + * future completed exceptionally + */ + public static R syncReturn(Class clazz) + throws Exception { + CompletableFuture completableFuture = CUR_COMPLETABLE_FUTURE.get(); + assert completableFuture != null; + try { + return (R) completableFuture.get(); + } catch (ExecutionException e) { + throw (Exception)e.getCause(); + } + } + + /** + * Completes the current asynchronous operation with the specified value. + * This method sets the result of the current thread's {@link CompletableFuture} + * to the provided value, effectively completing the asynchronous operation. + * + * @param value The value to complete the future with. + * @param The type of the value to be completed. + */ + public static void asyncComplete(R value) { + CUR_COMPLETABLE_FUTURE.set( + CompletableFuture.completedFuture(value)); + } + + /** + * Completes the current asynchronous operation with the specified completableFuture. + * + * @param completableFuture The completableFuture to complete the future with. + * @param The type of the value to be completed. + */ + public static void asyncCompleteWith(CompletableFuture completableFuture) { + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) completableFuture); + } + + public static CompletableFuture getAsyncUtilCompletableFuture() { + assert CUR_COMPLETABLE_FUTURE.get() != null; + return CUR_COMPLETABLE_FUTURE.get(); + } + + /** + * Completes the current asynchronous operation with an exception. + * This method sets the result of the current thread's {@link CompletableFuture} + * to an exceptional completion, using the provided {@link Throwable} as the cause. + * This is typically used to handle errors in asynchronous operations. + * + * @param e The exception to complete the future exceptionally with. + */ + public static void asyncThrowException(Throwable e) { + CompletableFuture result = new CompletableFuture<>(); + result.completeExceptionally(warpCompletionException(e)); + CUR_COMPLETABLE_FUTURE.set(result); + } + + /** + * Applies an asynchronous function to the current {@link CompletableFuture}. + * This method retrieves the current thread's {@link CompletableFuture} and applies + * the provided {@link ApplyFunction} to it. It is used to chain asynchronous + * operations, where the result of one operation is used as the input for the next. + * + * @param function The asynchronous function to apply, which takes a type T and + * produces a type R. + * @param The type of the input to the function. + * @param The type of the result of the function. + * @see CompletableFuture + * @see ApplyFunction + */ + public static void asyncApply(ApplyFunction function) { + CompletableFuture completableFuture = + (CompletableFuture) CUR_COMPLETABLE_FUTURE.get(); + assert completableFuture != null; + CompletableFuture result = function.apply(completableFuture); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } + + /** + * Applies an asynchronous function to the current {@link CompletableFuture} + * using the specified executor. This method retrieves the current thread's + * {@link CompletableFuture} and applies the provided{@link ApplyFunction} to + * it with the given executor service. It allows for more control over the + * execution context, such as running the operation in a separate thread or + * thread pool. + * + *

This is particularly useful when you need to perform blocking I/O operations + * or other long-running tasks without blocking the main thread or + * when you want to manage the thread resources more efficiently.

+ * + * @param function The asynchronous function to apply, which takes a type T and + * produces a type R. + * @param executor The executor service used to run the asynchronous function. + * @param The type of the input to the function. + * @param The type of the result of the function. + * @see CompletableFuture + * @see ApplyFunction + */ + public static void asyncApplyUseExecutor( + ApplyFunction function, Executor executor) { + CompletableFuture completableFuture = + (CompletableFuture) CUR_COMPLETABLE_FUTURE.get(); + assert completableFuture != null; + CompletableFuture result = function.apply(completableFuture, executor); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } + + /** + * Attempts to execute an asynchronous task defined by the provided + * {@link AsyncRun} and associates it with the current thread's + * {@link CompletableFuture}. This method is useful for trying operations + * that may throw exceptions and handling them asynchronously. + * + *

The provided {@code asyncRun} is a functional interface that + * encapsulates the logic to be executed asynchronously. It is executed in + * the context of the current CompletableFuture, allowing for chaining further + * asynchronous operations based on the result or exception of this try.

+ * + *

If the operation completes successfully, the result is propagated to the + * next operation in the chain. If an exception occurs, it can be caught and + * handled using the {@link #asyncCatch(CatchFunction, Class)} method, + * allowing for error recovery or alternative processing.

+ * + * @param asyncRun The asynchronous task to be executed, defined by + * an {@link AsyncRun} instance. + * @param The type of the result produced by the asynchronous task. + * @see AsyncRun + * @see #asyncCatch(CatchFunction, Class) + */ + public static void asyncTry(AsyncRun asyncRun) { + try { + CompletableFuture result = asyncRun.async(); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } catch (Throwable e) { + asyncThrowException(e); + } + } + + /** + * Handles exceptions to a specified type that may occur during + * an asynchronous operation. This method is used to catch and deal + * with exceptions in a non-blocking manner, allowing the application + * to continue processing even when errors occur. + * + *

The provided {@code function} is a {@link CatchFunction} that + * defines how to handle the caught exception. It takes the result of + * the asynchronous operation (if any) and the caught exception, and + * returns a new result or modified result to continue the asynchronous + * processing.

+ * + *

The {@code eClass} parameter specifies the type of exceptions to + * catch. Only exceptions that are instances of this type (or its + * subclasses) will be caught and handled by the provided function.

+ * + * @param function The {@link CatchFunction} that defines how to + * handle the caught exception. + * @param eClass The class of the exception type to catch. + * @param The type of the result of the asynchronous operation. + * @param The type of the exception to catch. + * @see CatchFunction + */ + public static void asyncCatch( + CatchFunction function, Class eClass) { + CompletableFuture completableFuture = + (CompletableFuture) CUR_COMPLETABLE_FUTURE.get(); + assert completableFuture != null; + CompletableFuture result = function.apply(completableFuture, eClass); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } + + /** + * Executes a final action after an asynchronous operation + * completes, regardless of whether the operation was successful + * or resulted in an exception. This method provides a way to + * perform cleanup or finalization tasks in an asynchronous + * workflow. + * + *

The provided {@code function} is a {@link FinallyFunction} + * that encapsulates the logic to be executed after the + * asynchronous operation. It takes the result of the operation + * and returns a new result, which can be used to continue the + * asynchronous processing or to handle the final output of + * the workflow.

+ * + *

This method is particularly useful for releasing resources, + * closing connections, or performing other cleanup actions that + * need to occur after all other operations have completed.

+ * + * @param function The {@link FinallyFunction} that defines + * the final action to be executed. + * @param The type of the result of the asynchronous + * operation. + * @see FinallyFunction + */ + public static void asyncFinally(FinallyFunction function) { + CompletableFuture completableFuture = + (CompletableFuture) CUR_COMPLETABLE_FUTURE.get(); + assert completableFuture != null; + CompletableFuture result = function.apply(completableFuture); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } + + /** + * Executes an asynchronous operation for each element in an Iterator, applying + * a given async function to each element. This method is part of the asynchronous + * utilities provided to facilitate non-blocking operations on collections of elements. + * + *

The provided {@code asyncDo} is an {@link AsyncBiFunction} that encapsulates + * the logic to be executed asynchronously for each element. It is executed in + * the context of the current CompletableFuture, allowing for chaining further + * asynchronous operations based on the result or exception of each iteration.

+ * + *

The method is particularly useful for performing asynchronous iterations + * over collections where the processing of each element is independent.

+ * + * @param forEach the Iterator over which to iterate and apply the async function + * @param asyncDo the asynchronous function to apply to each element of the Iterator, + * implemented as an {@link AsyncBiFunction} + * @param the type of the elements being iterated over + * @param the type of the result produced by the asynchronous task applied to each element + * @see AsyncBiFunction + * @see AsyncForEachRun + */ + public static void asyncForEach( + Iterator forEach, AsyncBiFunction, I, R> asyncDo) { + AsyncForEachRun asyncForEachRun = new AsyncForEachRun<>(); + asyncForEachRun.forEach(forEach).asyncDo(asyncDo).run(); + } + + /** + * Applies an asynchronous operation to each element of a collection + * and aggregates the results. This method is designed to process a + * collection of elements concurrently using asynchronous tasks, and + * then combine the results into a single aggregated result. + * + *

The operation defined by {@code asyncDo} is applied to each + * element of the collection. This operation is expected to return a + * {@link CompletableFuture} representing the asynchronous task. + * Once all tasks have been started, the method (async) waits for all of + * them to complete and then uses the {@code then} function to + * process and aggregate the results.

+ * + *

The {@code then} function takes an array of {@link CompletableFuture} + * instances, each representing the future result of an individual + * asynchronous operation. It should return a new aggregated result + * based on these futures. This allows for various forms of result + * aggregation, such as collecting all results into a list, + * reducing them to a single value, or performing any other custom + * aggregation logic.

+ * + * @param collection the collection of elements to process. + * @param asyncDo the asynchronous operation to apply to each + * element. It must return a {@link CompletableFuture} + * representing the operation. + * @param then a function that takes an array of futures + * representing the results of the asynchronous + * operations and returns an aggregated result. + * @param the type of the elements in the collection. + * @param the type of the intermediate result from the + * asynchronous operations. + * @param

the type of the final aggregated result. + * @see CompletableFuture + */ + public static void asyncCurrent( + Collection collection, AsyncApplyFunction asyncDo, + Function[], P> then) { + CompletableFuture[] completableFutures = + new CompletableFuture[collection.size()]; + int i = 0; + for(I entry : collection) { + CompletableFuture future = null; + try { + future = asyncDo.async(entry); + } catch (IOException e) { + future = new CompletableFuture<>(); + future.completeExceptionally(warpCompletionException(e)); + } + completableFutures[i++] = future; + } + CompletableFuture

result = CompletableFuture.allOf(completableFutures) + .handle((unused, throwable) -> then.apply(completableFutures)); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } + + /** + * Get the CompletableFuture object stored in the current thread's local variable. + * + * @return The completableFuture object. + */ + public static CompletableFuture getCompletableFuture() { + return CUR_COMPLETABLE_FUTURE.get(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/CatchFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/CatchFunction.java new file mode 100644 index 0000000000000..a87ddf18aaeab --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/CatchFunction.java @@ -0,0 +1,120 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.unWarpCompletionException; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; + +/** + * The {@code CatchFunction} interface represents a function that handles exceptions + * occurring within an asynchronous operation. It provides a mechanism to catch and + * process exceptions to a specific type, allowing for error recovery or alternative + * processing paths within an asynchronous workflow. + * + *

This interface is part of the asynchronous utilities provided by the Hadoop + * Distributed File System (HDFS) Federation router. It is used in conjunction with + * other asynchronous interfaces such as {@link AsyncRun} and + * {@link FinallyFunction} to build complex, non-blocking operations.

+ * + *

An implementation of this interface should define how to handle a caught + * exception. It takes two parameters: the result of the asynchronous operation (if + * any) and the caught exception. The function can then return a new result or modify + * the existing result to continue the asynchronous processing.

+ * + *

CatchFunction is used to implement the following semantics:

+ *
+ * {@code
+ *    try{
+ *      R res = doAsync(input);
+ *    } catch(E e) {
+ *      // Can use CatchFunction
+ *      R result = thenApply(res, e);
+ *    }
+ * }
+ * 
+ * + * @param the type of the result of the asynchronous operation + * @param the type of the exception to catch, extending {@link Throwable} + * @see AsyncRun + * @see FinallyFunction + */ +@FunctionalInterface +public interface CatchFunction + extends Async { + + /** + * Applies this catch function to the given result and exception. + *

+ * This method is called to process an exception that occurred during an asynchronous + * operation. The implementation of this method should define how to handle the + * caught exception. It may involve logging the error, performing a recovery operation, + * or any other custom error handling logic. + *

+ * The method takes two parameters: the result of the asynchronous operation (if any), + * and the caught exception. Depending on the implementation, the method may return a + * new result, modify the existing result, or throw a new exception. + * + * @param r the result of the asynchronous operation, which may be null if the operation + * did not complete successfully + * @param e the caught exception, which the function should handle + * @return the result after applying the catch function, which may be a new result or a + * modified version of the input result + * @throws IOException if an I/O error occurs during the application of the catch function + */ + R apply(R r, E e) throws IOException; + + /** + * Applies the catch function to a {@code CompletableFuture}, handling exceptions of a + * specified type. + *

+ * This default method provides a way to integrate exception handling into a chain of + * asynchronous operations. It takes a {@code CompletableFuture} and a class object + * representing the type of exception to catch. The method uses the handle method of the + * {@code CompletableFuture} to apply the catch function. + *

+ * If the input future completes exceptionally with an instance of the specified exception + * type, the catch function is applied to the exception. If the future completes with a + * different type of exception or normally, the original result or exception is propagated. + * + * @param in the input {@code CompletableFuture} to which the catch function is applied + * @param eClazz the class object representing the exception type to catch + * @return a new {@code CompletableFuture} that completes with the result of applying + * the catch function, or propagates the original exception if it does not match + * the specified type + */ + default CompletableFuture apply( + CompletableFuture in, Class eClazz) { + return in.handle((r, e) -> { + if (e == null) { + return r; + } + Throwable readException = unWarpCompletionException(e); + if (eClazz.isInstance(readException)) { + try { + return CatchFunction.this.apply(r, (E) readException); + } catch (IOException ioe) { + throw warpCompletionException(ioe); + } + } + throw warpCompletionException(e); + }); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/FinallyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/FinallyFunction.java new file mode 100644 index 0000000000000..671d380ac8bd8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/FinallyFunction.java @@ -0,0 +1,96 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.Async.warpCompletionException; + +/** + * The {@code FinallyFunction} interface represents a function that is used to perform + * final actions after an asynchronous operation completes, regardless of whether the + * operation was successful or resulted in an exception. This interface is part of + * the asynchronous utilities provided by the Hadoop Distributed File System (HDFS) + * Federation router. + * + *

A {@code FinallyFunction} is typically used for cleanup or finalization tasks, + * such as releasing resources, closing connections, or performing other actions that + * need to occur after all other operations have completed.

+ * + *

An implementation of this interface should define what the final action is. It + * takes the result of the asynchronous operation as an argument and returns a new + * result, which can be the same as the input result or a modified version of it.

+ * + *

FinallyFunction is used to implement the following semantics:

+ *
+ * {@code
+ *    try{
+ *      R res = doAsync(input);
+ *    } catch(...) {
+ *      ...
+ *    } finally {
+ *      // Can use FinallyFunction
+ *      R result = thenApply(res);
+ *    }
+ * }
+ * 
+ * + * @param the type of the result of the asynchronous operation + */ +@FunctionalInterface +public interface FinallyFunction { + + /** + * Applies this final action function to the result of an asynchronous operation. + * + * @param r the result of the asynchronous operation, which may be null if the + * operation did not complete successfully + * @return the result after applying the final action, which may be a new result or a + * modified version of the input result + * @throws IOException if an I/O error occurs during the application of the final action + */ + R apply(R r) throws IOException; + + /** + * Applies this final action function to a {@code CompletableFuture}, which is expected + * to be the result of an asynchronous operation. + *

+ * This method is a convenience that simplifies the use of {@code FinallyFunction} + * with asynchronous operations. It handles the completion of the future and applies + * the {@code FinallyFunction} to the result. + * + * @param in the {@code CompletableFuture} representing the asynchronous operation + * @return a new {@code CompletableFuture} that completes with the result of applying + * the final action function + */ + default CompletableFuture apply(CompletableFuture in) { + return in.handle((r, e) -> { + try { + R ret = apply(r); + if (e != null) { + throw warpCompletionException(e); + } else { + return ret; + } + } catch (IOException ioe) { + throw warpCompletionException(ioe); + } + }); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/package-info.java new file mode 100644 index 0000000000000..5ffbebf9e71f3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/package-info.java @@ -0,0 +1,35 @@ +/** + * 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. + */ + +/** + * This package contains classes that facilitate asynchronous operations within the Hadoop + * Distributed File System (HDFS) Federation router. These classes are designed to work with + * the Hadoop ecosystem, providing utilities and interfaces to perform non-blocking tasks that + * can improve the performance and responsiveness of HDFS operations. + * + *

These classes work together to enable complex asynchronous workflows, making it easier to + * write code that can handle long-running tasks without blocking, thus improving the overall + * efficiency and scalability of HDFS operations.

+ */ +@InterfaceAudience.Private +@InterfaceStability.Evolving + +package org.apache.hadoop.hdfs.server.federation.router.async.utils; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml index 26b89ce0313fd..323aef952f341 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml @@ -48,6 +48,41 @@ + + dfs.federation.router.async.rpc.enable + false + + If true, router will process the RPC request asynchronously. + + + + + dfs.federation.router.async.rpc.ns.handler.count + + + The number of asynchronous handlers per nameservice, separated by commas, internally separated by colons. + The identifier of nameservice is in dfs.nameservices configuration entry. + Such as: ns1:count1,ns2:count2,ns3:count3. + + + + + dfs.federation.router.async.rpc.responder.count + 10 + + For those nameservices not in dfs.federation.router.async.rpc.ns.handler.count configuration entry, + use this value as the asynchronous handler thread counts. + + + + + dfs.federation.router.async.rpc.responder.count + 10 + + The thread counts of async responder executor. + + + dfs.federation.router.rpc-address 0.0.0.0:8888 @@ -101,6 +136,22 @@ + + dfs.federation.router.async.rpc.handler.count + 10 + + The number of async handler for the router to handle RPC client requests. + + + + + dfs.federation.router.async.rpc.responder.count + 10 + + The number of async responder for the router to handle responses. + + + dfs.federation.router.connection.creator.queue-size 100 diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestAsyncRpcProtocolPBUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestAsyncRpcProtocolPBUtil.java new file mode 100644 index 0000000000000..b696572dd4787 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestAsyncRpcProtocolPBUtil.java @@ -0,0 +1,123 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.ProtobufRpcEngine2; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.ipc.TestRPC; +import org.apache.hadoop.ipc.TestRpcBase; +import org.apache.hadoop.ipc.protobuf.TestRpcServiceProtos; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.thirdparty.protobuf.BlockingService; +import org.apache.hadoop.util.Time; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.ForkJoinPool; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class TestAsyncRpcProtocolPBUtil { + private static final Logger LOG = LoggerFactory.getLogger(TestAsyncRpcProtocolPBUtil.class); + private static final int SERVER_PROCESS_COST_MS = 100; + private TestClientProtocolTranslatorPB clientPB; + private Server rpcServer; + + @Before + public void setUp() throws IOException { + AsyncRpcProtocolPBUtil.setAsyncResponderExecutor(ForkJoinPool.commonPool()); + Configuration conf = new Configuration(); + RPC.setProtocolEngine(conf, TestRpcBase.TestRpcService.class, + ProtobufRpcEngine2.class); + + // Create server side implementation + TestClientProtocolServerSideTranslatorPB serverImpl = + new TestClientProtocolServerSideTranslatorPB(SERVER_PROCESS_COST_MS); + BlockingService service = TestRpcServiceProtos.TestProtobufRpcProto + .newReflectiveBlockingService(serverImpl); + + // start the IPC server + rpcServer = new RPC.Builder(conf) + .setProtocol(TestRpcBase.TestRpcService.class) + .setInstance(service).setBindAddress("0.0.0.0") + .setPort(0).setNumHandlers(1).setVerbose(true).build(); + + rpcServer.start(); + + InetSocketAddress addr = NetUtils.getConnectAddress(rpcServer); + + TestRpcBase.TestRpcService proxy = RPC.getProxy(TestRpcBase.TestRpcService.class, + TestRPC.TestProtocol.versionID, addr, conf); + clientPB = new TestClientProtocolTranslatorPB(proxy); + Client.setAsynchronousMode(true); + clientPB.ping(); + } + + @After + public void clear() { + if (clientPB != null) { + clientPB.close(); + } + if (rpcServer != null) { + rpcServer.stop(); + } + } + + @Test + public void testAsyncIpcClient() throws Exception { + Client.setAsynchronousMode(true); + long start = Time.monotonicNow(); + clientPB.add(1, 2); + long cost = Time.monotonicNow() - start; + LOG.info("rpc client add {} {}, cost: {}ms", 1, 2, cost); + Integer res = syncReturn(Integer.class); + checkResult(3, res, cost); + + start = Time.monotonicNow(); + clientPB.echo("test echo!"); + cost = Time.monotonicNow() - start; + LOG.info("rpc client echo {}, cost: {}ms", "test echo!", cost); + String value = syncReturn(String.class); + checkResult("test echo!", value, cost); + + start = Time.monotonicNow(); + clientPB.error(); + LOG.info("rpc client error, cost: {}ms", Time.monotonicNow() - start); + LambdaTestUtils.intercept(RemoteException.class, "test!", + () -> AsyncUtil.syncReturn(String.class)); + } + + private void checkResult(Object expected, Object actual, long cost) { + assertTrue(cost < SERVER_PROCESS_COST_MS); + assertEquals(expected, actual); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocol.java new file mode 100644 index 0000000000000..fee964a529270 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocol.java @@ -0,0 +1,31 @@ +/** + * 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; + +public interface TestClientProtocol { + void ping() throws IOException; + + String echo(String echoMessage) throws IOException; + + void error() throws IOException; + + int add(int num1, int num2) throws IOException; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocolServerSideTranslatorPB.java new file mode 100644 index 0000000000000..4e64a3af9dddb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocolServerSideTranslatorPB.java @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.ipc.StandbyException; +import org.apache.hadoop.ipc.TestRpcBase; +import org.apache.hadoop.ipc.protobuf.TestProtos; +import org.apache.hadoop.thirdparty.protobuf.RpcController; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; +import org.apache.hadoop.util.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestClientProtocolServerSideTranslatorPB extends TestRpcBase.PBServerImpl { + private static final Logger LOG = + LoggerFactory.getLogger(TestClientProtocolServerSideTranslatorPB.class); + private final int processTime; + + public TestClientProtocolServerSideTranslatorPB(int processTime) { + this.processTime = processTime; + } + + @Override + public TestProtos.EmptyResponseProto error( + RpcController unused, TestProtos.EmptyRequestProto request) + throws ServiceException { + long start = Time.monotonicNow(); + try { + Thread.sleep(processTime); + throw new ServiceException("error", new StandbyException("test!")); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } finally { + LOG.info("rpc server error cost: {}ms", Time.monotonicNow() - start); + } + return null; + } + + @Override + public TestProtos.EchoResponseProto echo( + RpcController unused, TestProtos.EchoRequestProto request) throws ServiceException { + TestProtos.EchoResponseProto res = null; + long start = Time.monotonicNow(); + try { + Thread.sleep(processTime); + res = super.echo(unused, request); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } finally { + LOG.info("rpc server echo: {}, result: {}, cost: {}ms", request.getMessage(), + res.getMessage(), Time.monotonicNow() - start); + } + return res; + } + + @Override + public TestProtos.AddResponseProto add( + RpcController controller, TestProtos.AddRequestProto request) throws ServiceException { + TestProtos.AddResponseProto res = null; + long start = Time.monotonicNow(); + try { + Thread.sleep(processTime); + res = super.add(controller, request); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } finally { + LOG.info("rpc server add: {} {}, result: {}, cost: {}ms", + request.getParam1(), request.getParam2(), res.getResult(), Time.monotonicNow() - start); + } + return res; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocolTranslatorPB.java new file mode 100644 index 0000000000000..3fd9a2c4ea0b6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestClientProtocolTranslatorPB.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.TestRpcBase.TestRpcService; +import org.apache.hadoop.ipc.protobuf.TestProtos.EmptyRequestProto; +import org.apache.hadoop.ipc.protobuf.TestProtos.EchoRequestProto; +import org.apache.hadoop.ipc.protobuf.TestProtos.AddRequestProto; + +import java.io.Closeable; +import java.io.IOException; + +public class TestClientProtocolTranslatorPB implements TestClientProtocol, Closeable { + final private TestRpcService rpcProxy; + + public TestClientProtocolTranslatorPB(TestRpcService rpcProxy) { + this.rpcProxy = rpcProxy; + } + + @Override + public void ping() throws IOException { + EmptyRequestProto req = EmptyRequestProto.newBuilder() + .build(); + + AsyncRpcProtocolPBUtil.asyncIpcClient(() -> rpcProxy.ping(null, req), + res -> null, Void.class); + } + + @Override + public String echo(String echoMessage) throws IOException { + EchoRequestProto req = EchoRequestProto.newBuilder() + .setMessage(echoMessage) + .build(); + + return AsyncRpcProtocolPBUtil.asyncIpcClient(() -> rpcProxy.echo(null, req), + res -> res.getMessage(), String.class); + } + + @Override + public void error() throws IOException { + EmptyRequestProto req = EmptyRequestProto.newBuilder() + .build(); + + AsyncRpcProtocolPBUtil.asyncIpcClient(() -> rpcProxy.error(null, req), + res -> null, Void.class); + } + + @Override + public int add(int num1, int num2) throws IOException { + AddRequestProto req = AddRequestProto.newBuilder() + .setParam1(num1) + .setParam2(num2) + .build(); + + return AsyncRpcProtocolPBUtil.asyncIpcClient(() -> rpcProxy.add(null, req), + res -> res.getResult(), Integer.class); + } + + @Override + public void close() { + RPC.stopProxy(rpcProxy); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestRouterClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestRouterClientSideTranslatorPB.java new file mode 100644 index 0000000000000..4732dc778b5f5 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestRouterClientSideTranslatorPB.java @@ -0,0 +1,244 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.protocolPB; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.crypto.CryptoProtocolVersion; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; +import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; +import org.apache.hadoop.io.EnumSetWritable; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.ProtobufRpcEngine2; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolPB; +import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB; +import org.apache.hadoop.util.Lists; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.EnumSet; +import java.util.List; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.hadoop.crypto.CryptoProtocolVersion.ENCRYPTION_ZONES; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import static org.apache.hadoop.fs.permission.AclEntryType.USER; +import static org.apache.hadoop.fs.permission.FsAction.ALL; +import static org.apache.hadoop.fs.permission.FsAction.NONE; +import static org.apache.hadoop.fs.permission.FsAction.READ; +import static org.apache.hadoop.fs.permission.FsAction.READ_WRITE; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class TestRouterClientSideTranslatorPB { + private static MiniDFSCluster cluster = null; + private static InetSocketAddress nnAddress = null; + private static Configuration conf = null; + private static RouterClientProtocolTranslatorPB clientProtocolTranslatorPB; + private static RouterGetUserMappingsProtocolTranslatorPB getUserMappingsProtocolTranslatorPB; + private static RouterNamenodeProtocolTranslatorPB namenodeProtocolTranslatorPB; + private static RouterRefreshUserMappingsProtocolTranslatorPB + refreshUserMappingsProtocolTranslatorPB; + private static final String TEST_DIR_PATH = "/test"; + private boolean mode; + + @BeforeClass + public static void setUp() throws Exception { + AsyncRpcProtocolPBUtil.setAsyncResponderExecutor(ForkJoinPool.commonPool()); + conf = new HdfsConfiguration(); + cluster = (new MiniDFSCluster.Builder(conf)) + .numDataNodes(1).build(); + cluster.waitClusterUp(); + nnAddress = cluster.getNameNode().getNameNodeAddress(); + clientProtocolTranslatorPB = new RouterClientProtocolTranslatorPB( + createProxy(ClientNamenodeProtocolPB.class)); + getUserMappingsProtocolTranslatorPB = new RouterGetUserMappingsProtocolTranslatorPB( + createProxy(GetUserMappingsProtocolPB.class)); + namenodeProtocolTranslatorPB = new RouterNamenodeProtocolTranslatorPB( + createProxy(NamenodeProtocolPB.class)); + refreshUserMappingsProtocolTranslatorPB = new RouterRefreshUserMappingsProtocolTranslatorPB( + createProxy(RefreshUserMappingsProtocolPB.class)); + } + + @AfterClass + public static void tearDown() throws Exception { + if (clientProtocolTranslatorPB != null) { + clientProtocolTranslatorPB.close(); + } + if (getUserMappingsProtocolTranslatorPB != null) { + getUserMappingsProtocolTranslatorPB.close(); + } + if (namenodeProtocolTranslatorPB != null) { + namenodeProtocolTranslatorPB.close(); + } + if (refreshUserMappingsProtocolTranslatorPB != null) { + refreshUserMappingsProtocolTranslatorPB.close(); + } + if (cluster != null) { + cluster.shutdown(); + } + } + + @Before + public void setAsync() { + mode = Client.isAsynchronousMode(); + Client.setAsynchronousMode(true); + } + + @After + public void unsetAsync() { + Client.setAsynchronousMode(mode); + } + + @Test + public void testRouterClientProtocolTranslatorPB() throws Exception { + clientProtocolTranslatorPB.mkdirs(TEST_DIR_PATH, new FsPermission(ALL, ALL, ALL), false); + Boolean success = syncReturn(Boolean.class); + assertTrue(success); + + clientProtocolTranslatorPB.setPermission(TEST_DIR_PATH, + new FsPermission(READ_WRITE, READ, NONE)); + syncReturn(Void.class); + + clientProtocolTranslatorPB.getFileInfo(TEST_DIR_PATH); + HdfsFileStatus hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertEquals(hdfsFileStatus.getPermission(), new FsPermission(READ_WRITE, READ, NONE)); + + List aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "tmpUser", ALL)); + clientProtocolTranslatorPB.setAcl(TEST_DIR_PATH, aclSpec); + syncReturn(Void.class); + clientProtocolTranslatorPB.setOwner(TEST_DIR_PATH, "tmpUser", "tmpUserGroup"); + syncReturn(Void.class); + + clientProtocolTranslatorPB.getFileInfo(TEST_DIR_PATH); + hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertEquals("tmpUser", hdfsFileStatus.getOwner()); + assertEquals("tmpUserGroup", hdfsFileStatus.getGroup()); + + clientProtocolTranslatorPB.create(TEST_DIR_PATH + "/testCreate.file", + new FsPermission(ALL, ALL, ALL), "testAsyncClient", + new EnumSetWritable<>(EnumSet.of(CreateFlag.CREATE)), + false, (short) 1, 128 * 1024 * 1024L, new CryptoProtocolVersion[]{ENCRYPTION_ZONES}, + null, null); + hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertTrue(hdfsFileStatus.isFile()); + assertEquals(128 * 1024 * 1024, hdfsFileStatus.getBlockSize()); + + clientProtocolTranslatorPB.getListing(TEST_DIR_PATH, new byte[1], true); + DirectoryListing directoryListing = syncReturn(DirectoryListing.class); + assertEquals(1, directoryListing.getPartialListing().length); + + clientProtocolTranslatorPB.getDatanodeReport(HdfsConstants.DatanodeReportType.ALL); + DatanodeInfo[] datanodeInfos = syncReturn(DatanodeInfo[].class); + assertEquals(1, datanodeInfos.length); + + clientProtocolTranslatorPB.createSymlink(TEST_DIR_PATH + "/testCreate.file", + "/link/link.file", new FsPermission(ALL, ALL, ALL), true); + syncReturn(Void.class); + + clientProtocolTranslatorPB.getFileLinkInfo("/link/link.file"); + hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertEquals("testCreate.file", hdfsFileStatus.getSymlink().getName()); + + clientProtocolTranslatorPB.rename(TEST_DIR_PATH + "/testCreate.file", + TEST_DIR_PATH + "/testRename.file"); + success = syncReturn(boolean.class); + assertTrue(success); + + clientProtocolTranslatorPB.delete(TEST_DIR_PATH, true); + success = syncReturn(boolean.class); + assertTrue(success); + + LambdaTestUtils.intercept(RemoteException.class, "Parent directory doesn't exist: /test", + () -> { + clientProtocolTranslatorPB.mkdirs(TEST_DIR_PATH + "/testCreate.file", + new FsPermission(ALL, ALL, ALL), false); + syncReturn(boolean.class); + }); + } + + @Test + public void testRouterGetUserMappingsProtocolTranslatorPB() throws Exception { + getUserMappingsProtocolTranslatorPB.getGroupsForUser("root"); + String[] strings = syncReturn(String[].class); + assertTrue(strings.length != 0); + + getUserMappingsProtocolTranslatorPB.getGroupsForUser("tmp"); + strings = syncReturn(String[].class); + assertEquals(0, strings.length); + } + + @Test + public void testRouterNamenodeProtocolTranslatorPB() throws Exception { + namenodeProtocolTranslatorPB.getTransactionID(); + Long id = syncReturn(Long.class); + assertTrue(id > 0); + + namenodeProtocolTranslatorPB.getBlockKeys(); + ExportedBlockKeys exportedBlockKeys = syncReturn(ExportedBlockKeys.class); + assertNotNull(exportedBlockKeys); + + namenodeProtocolTranslatorPB.rollEditLog(); + CheckpointSignature checkpointSignature = syncReturn(CheckpointSignature.class); + assertNotNull(checkpointSignature); + } + + @Test + public void testRouterRefreshUserMappingsProtocolTranslatorPB() throws Exception { + refreshUserMappingsProtocolTranslatorPB.refreshUserToGroupsMappings(); + syncReturn(Void.class); + + refreshUserMappingsProtocolTranslatorPB.refreshSuperUserGroupsConfiguration(); + syncReturn(Void.class); + } + + public static

P createProxy(Class

protocol) throws IOException { + RPC.setProtocolEngine( + conf, protocol, ProtobufRpcEngine2.class); + final long version = RPC.getProtocolVersion(protocol); + return RPC.getProtocolProxy( + protocol, version, nnAddress, UserGroupInformation.getCurrentUser(), + conf, + NetUtils.getDefaultSocketFactory(conf), + RPC.getRpcTimeout(conf), null, + new AtomicBoolean(false)).getProxy(); + } +} 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 7edb549233062..39b132ce31539 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 @@ -163,12 +163,12 @@ public class TestRouterRpc { private static final Logger LOG = LoggerFactory.getLogger(TestRouterRpc.class); - private static final int NUM_SUBCLUSTERS = 2; + protected static final int NUM_SUBCLUSTERS = 2; // We need at least 6 DNs to test Erasure Coding with RS-6-3-64k - private static final int NUM_DNS = 6; + protected static final int NUM_DNS = 6; - private static final Comparator EC_POLICY_CMP = + protected static final Comparator EC_POLICY_CMP = new Comparator() { public int compare( ErasureCodingPolicyInfo ec0, @@ -214,6 +214,18 @@ public int compare( @BeforeClass public static void globalSetUp() throws Exception { + // Start routers with only an RPC service + Configuration routerConf = new RouterConfigBuilder() + .metrics() + .rpc() + .build(); + // We decrease the DN cache times to make the test faster + routerConf.setTimeDuration( + RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); + setUp(routerConf); + } + + public static void setUp(Configuration routerConf) throws Exception { Configuration namenodeConf = new Configuration(); namenodeConf.setBoolean(DFSConfigKeys.HADOOP_CALLER_CONTEXT_ENABLED_KEY, true); @@ -242,14 +254,6 @@ public static void globalSetUp() throws Exception { // Start NNs and DNs and wait until ready cluster.startCluster(); - // Start routers with only an RPC service - Configuration routerConf = new RouterConfigBuilder() - .metrics() - .rpc() - .build(); - // We decrease the DN cache times to make the test faster - routerConf.setTimeDuration( - RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); cluster.addRouterOverrides(routerConf); cluster.startRouters(); @@ -1260,8 +1264,8 @@ public void testProxyConcatFile() throws Exception { createFile(routerFS, targetFile, existingFileSize); // Concat in same namespaces, succeeds testConcat(srcEmptyFile, targetFile, true, true, - "org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.HadoopIllegalArgumentException): concat: source file " - + srcEmptyFile + " is invalid or empty or underConstruction"); + "org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.HadoopIllegalArgumentException): " + + "concat: source file " + srcEmptyFile + " is invalid or empty or underConstruction"); } @Test @@ -2023,7 +2027,7 @@ public void testCacheAdmin() throws Exception { } @Test - public void testgetGroupsForUser() throws IOException { + public void testgetGroupsForUser() throws Exception { String[] group = new String[] {"bar", "group2"}; UserGroupInformation.createUserForTesting("user", new String[] {"bar", "group2"}); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncProtocolTestBase.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncProtocolTestBase.java new file mode 100644 index 0000000000000..893afddbc635a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncProtocolTestBase.java @@ -0,0 +1,165 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; +import org.apache.hadoop.hdfs.server.federation.MockResolver; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; +import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.ipc.CallerContext; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES; +import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.DEFAULT_HEARTBEAT_INTERVAL_MS; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY; +import static org.junit.Assert.assertTrue; + +/** + * Used to test the functionality of async router rps. + */ +public class RouterAsyncProtocolTestBase { + private static Configuration routerConf; + /** Federated HDFS cluster. */ + private static MiniRouterDFSCluster cluster; + private static String ns0; + + /** Random Router for this federated cluster. */ + private MiniRouterDFSCluster.RouterContext router; + private FileSystem routerFs; + private RouterRpcServer routerRpcServer; + private RouterRpcServer routerAsyncRpcServer; + protected static final String TEST_DIR_PATH = "/testdir"; + + @BeforeClass + public static void setUpCluster() throws Exception { + cluster = new MiniRouterDFSCluster(true, 1, 2, + DEFAULT_HEARTBEAT_INTERVAL_MS, 1000); + cluster.setNumDatanodesPerNameservice(3); + + cluster.startCluster(); + + // Making one Namenode active per nameservice + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + cluster.switchToActive(ns, NAMENODES[0]); + cluster.switchToStandby(ns, NAMENODES[1]); + } + } + // Start routers with only an RPC service + routerConf = new RouterConfigBuilder() + .rpc() + .build(); + + // Reduce the number of RPC clients threads to overload the Router easy + routerConf.setInt(RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY, 1); + // We decrease the DN cache times to make the test faster + routerConf.setTimeDuration( + RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); + cluster.addRouterOverrides(routerConf); + // Start routers with only an RPC service + cluster.startRouters(); + + // Register and verify all NNs with all routers + cluster.registerNamenodes(); + cluster.waitNamenodeRegistration(); + cluster.waitActiveNamespaces(); + ns0 = cluster.getNameservices().get(0); + } + + @AfterClass + public static void shutdownCluster() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Before + public void setUp() throws IOException { + router = cluster.getRandomRouter(); + routerFs = router.getFileSystem(); + routerRpcServer = router.getRouterRpcServer(); + routerRpcServer.initAsyncThreadPools(routerConf); + RouterAsyncRpcClient asyncRpcClient = new RouterAsyncRpcClient( + routerConf, router.getRouter(), routerRpcServer.getNamenodeResolver(), + routerRpcServer.getRPCMonitor(), + routerRpcServer.getRouterStateIdContext()); + routerAsyncRpcServer = Mockito.spy(routerRpcServer); + Mockito.when(routerAsyncRpcServer.getRPCClient()).thenReturn(asyncRpcClient); + Mockito.when(routerAsyncRpcServer.isAsync()).thenReturn(true); + + // Create mock locations + MockResolver resolver = (MockResolver) router.getRouter().getSubclusterResolver(); + resolver.addLocation("/", ns0, "/"); + FsPermission permission = new FsPermission("705"); + routerFs.mkdirs(new Path(TEST_DIR_PATH), permission); + } + + @After + public void tearDown() throws IOException { + // clear client context + CallerContext.setCurrent(null); + boolean delete = routerFs.delete(new Path(TEST_DIR_PATH)); + assertTrue(delete); + if (routerFs != null) { + routerFs.close(); + } + } + + public static Configuration getRouterConf() { + return routerConf; + } + + public static MiniRouterDFSCluster getCluster() { + return cluster; + } + + public static String getNs0() { + return ns0; + } + + public MiniRouterDFSCluster.RouterContext getRouter() { + return router; + } + + public FileSystem getRouterFs() { + return routerFs; + } + + public RouterRpcServer getRouterRpcServer() { + return routerRpcServer; + } + + public RouterRpcServer getRouterAsyncRpcServer() { + return routerAsyncRpcServer; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncCacheAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncCacheAdmin.java new file mode 100644 index 0000000000000..ef82ec267ebec --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncCacheAdmin.java @@ -0,0 +1,102 @@ +/** + * 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.async; + +import org.apache.hadoop.fs.CacheFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +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.fs.BatchedRemoteIterator.BatchedEntries; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.EnumSet; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertEquals; + +/** + * Used to test the functionality of {@link RouterAsyncCacheAdmin}. + */ +public class TestRouterAsyncCacheAdmin extends RouterAsyncProtocolTestBase { + private RouterAsyncCacheAdmin asyncCacheAdmin; + + @Before + public void setup() throws IOException { + asyncCacheAdmin = new RouterAsyncCacheAdmin(getRouterAsyncRpcServer()); + FSDataOutputStream fsDataOutputStream = getRouterFs().create( + new Path("/testCache.file"), true); + fsDataOutputStream.write(new byte[1024]); + fsDataOutputStream.close(); + } + + @Test + public void testRouterAsyncCacheAdmin() throws Exception { + asyncCacheAdmin.addCachePool(new CachePoolInfo("pool")); + syncReturn(null); + + CacheDirectiveInfo path = new CacheDirectiveInfo.Builder(). + setPool("pool"). + setPath(new Path("/testCache.file")). + build(); + asyncCacheAdmin.addCacheDirective(path, EnumSet.of(CacheFlag.FORCE)); + long result = syncReturn(long.class); + assertEquals(1, result); + + asyncCacheAdmin.listCachePools(""); + BatchedEntries cachePoolEntries = syncReturn(BatchedEntries.class); + assertEquals("pool", cachePoolEntries.get(0).getInfo().getPoolName()); + + CacheDirectiveInfo filter = new CacheDirectiveInfo.Builder(). + setPool("pool"). + build(); + asyncCacheAdmin.listCacheDirectives(0, filter); + BatchedEntries cacheDirectiveEntries = syncReturn(BatchedEntries.class); + assertEquals(new Path("/testCache.file"), cacheDirectiveEntries.get(0).getInfo().getPath()); + + CachePoolInfo pool = new CachePoolInfo("pool").setOwnerName("pool_user"); + asyncCacheAdmin.modifyCachePool(pool); + syncReturn(null); + + asyncCacheAdmin.listCachePools(""); + cachePoolEntries = syncReturn(BatchedEntries.class); + assertEquals("pool_user", cachePoolEntries.get(0).getInfo().getOwnerName()); + + path = new CacheDirectiveInfo.Builder(). + setPool("pool"). + setPath(new Path("/testCache.file")). + setReplication((short) 2). + setId(1L). + build(); + asyncCacheAdmin.modifyCacheDirective(path, EnumSet.of(CacheFlag.FORCE)); + syncReturn(null); + + asyncCacheAdmin.listCacheDirectives(0, filter); + cacheDirectiveEntries = syncReturn(BatchedEntries.class); + assertEquals(Short.valueOf((short) 2), cacheDirectiveEntries.get(0).getInfo().getReplication()); + + asyncCacheAdmin.removeCacheDirective(1L); + syncReturn(null); + asyncCacheAdmin.removeCachePool("pool"); + syncReturn(null); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncClientProtocol.java new file mode 100644 index 0000000000000..96f957f93df3e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncClientProtocol.java @@ -0,0 +1,144 @@ +/** + * 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.async; + +import org.apache.hadoop.crypto.CryptoProtocolVersion; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FsServerDefaults; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.federation.router.RouterClientProtocol; +import org.apache.hadoop.io.EnumSetWritable; +import org.apache.hadoop.util.Lists; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.List; + +import static org.apache.hadoop.crypto.CryptoProtocolVersion.ENCRYPTION_ZONES; +import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT; +import static org.apache.hadoop.fs.permission.AclEntryType.USER; +import static org.apache.hadoop.fs.permission.FsAction.ALL; +import static org.apache.hadoop.fs.permission.FsAction.NONE; +import static org.apache.hadoop.fs.permission.FsAction.READ; +import static org.apache.hadoop.fs.permission.FsAction.READ_WRITE; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Used to test the functionality of {@link RouterAsyncClientProtocol}. + */ +public class TestRouterAsyncClientProtocol extends RouterAsyncProtocolTestBase { + private RouterAsyncClientProtocol asyncClientProtocol; + private RouterClientProtocol clientProtocol; + private final String testPath = TEST_DIR_PATH + "/test"; + + @Before + public void setup() throws IOException { + asyncClientProtocol = new RouterAsyncClientProtocol(getRouterConf(), getRouterAsyncRpcServer()); + clientProtocol = new RouterClientProtocol(getRouterConf(), getRouterRpcServer()); + } + + @Test + public void testGetServerDefaults() throws Exception { + FsServerDefaults serverDefaults = clientProtocol.getServerDefaults(); + asyncClientProtocol.getServerDefaults(); + FsServerDefaults fsServerDefaults = syncReturn(FsServerDefaults.class); + assertEquals(serverDefaults.getBlockSize(), fsServerDefaults.getBlockSize()); + assertEquals(serverDefaults.getReplication(), fsServerDefaults.getReplication()); + assertEquals(serverDefaults.getChecksumType(), fsServerDefaults.getChecksumType()); + assertEquals( + serverDefaults.getDefaultStoragePolicyId(), fsServerDefaults.getDefaultStoragePolicyId()); + } + + @Test + public void testClientProtocolRpc() throws Exception { + asyncClientProtocol.mkdirs(testPath, new FsPermission(ALL, ALL, ALL), false); + Boolean success = syncReturn(Boolean.class); + assertTrue(success); + + asyncClientProtocol.setPermission(testPath, new FsPermission(READ_WRITE, READ, NONE)); + syncReturn(Void.class); + + asyncClientProtocol.getFileInfo(testPath); + HdfsFileStatus hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertEquals(hdfsFileStatus.getPermission(), new FsPermission(READ_WRITE, READ, NONE)); + + List aclSpec = Lists.newArrayList(aclEntry(DEFAULT, USER, "tmpUser", ALL)); + asyncClientProtocol.setAcl(testPath, aclSpec); + syncReturn(Void.class); + asyncClientProtocol.setOwner(testPath, "tmpUser", "tmpUserGroup"); + syncReturn(Void.class); + + asyncClientProtocol.getFileInfo(testPath); + hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertEquals("tmpUser", hdfsFileStatus.getOwner()); + assertEquals("tmpUserGroup", hdfsFileStatus.getGroup()); + + asyncClientProtocol.create(testPath + "/testCreate.file", + new FsPermission(ALL, ALL, ALL), "testAsyncClient", + new EnumSetWritable<>(EnumSet.of(CreateFlag.CREATE)), + false, (short) 1, 128 * 1024 * 1024L, + new CryptoProtocolVersion[]{ENCRYPTION_ZONES}, + null, null); + hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertTrue(hdfsFileStatus.isFile()); + assertEquals(128 * 1024 * 1024, hdfsFileStatus.getBlockSize()); + + asyncClientProtocol.getFileRemoteLocation(testPath); + RemoteLocation remoteLocation = syncReturn(RemoteLocation.class); + assertNotNull(remoteLocation); + assertEquals(getNs0(), remoteLocation.getNameserviceId()); + assertEquals(testPath, remoteLocation.getSrc()); + + asyncClientProtocol.getListing(testPath, new byte[1], true); + DirectoryListing directoryListing = syncReturn(DirectoryListing.class); + assertEquals(1, directoryListing.getPartialListing().length); + + asyncClientProtocol.getDatanodeReport(HdfsConstants.DatanodeReportType.ALL); + DatanodeInfo[] datanodeInfos = syncReturn(DatanodeInfo[].class); + assertEquals(3, datanodeInfos.length); + + asyncClientProtocol.createSymlink(testPath + "/testCreate.file", + "/link/link.file", new FsPermission(ALL, ALL, ALL), true); + syncReturn(Void.class); + + asyncClientProtocol.getFileLinkInfo("/link/link.file"); + hdfsFileStatus = syncReturn(HdfsFileStatus.class); + assertEquals("testCreate.file", hdfsFileStatus.getSymlink().getName()); + + asyncClientProtocol.rename(testPath + "/testCreate.file", + testPath + "/testRename.file"); + success = syncReturn(boolean.class); + assertTrue(success); + + asyncClientProtocol.delete(testPath, true); + success = syncReturn(boolean.class); + assertTrue(success); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncErasureCoding.java new file mode 100644 index 0000000000000..1cac778a8099f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncErasureCoding.java @@ -0,0 +1,206 @@ +/** + * 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.async; + + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; +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.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; +import org.apache.hadoop.hdfs.server.federation.MockResolver; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; +import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.ipc.CallerContext; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES; +import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.DEFAULT_HEARTBEAT_INTERVAL_MS; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class TestRouterAsyncErasureCoding { + private static Configuration routerConf; + /** Federated HDFS cluster. */ + private static MiniRouterDFSCluster cluster; + private static String ns0; + + /** Random Router for this federated cluster. */ + private MiniRouterDFSCluster.RouterContext router; + private FileSystem routerFs; + private RouterRpcServer routerRpcServer; + private AsyncErasureCoding asyncErasureCoding; + + private final String testfilePath = "/testdir/testAsyncErasureCoding.file"; + + @BeforeClass + public static void setUpCluster() throws Exception { + cluster = new MiniRouterDFSCluster(true, 1, 2, + DEFAULT_HEARTBEAT_INTERVAL_MS, 1000); + cluster.setNumDatanodesPerNameservice(3); + cluster.setRacks( + new String[] {"/rack1", "/rack2", "/rack3"}); + cluster.startCluster(); + + // Making one Namenode active per nameservice + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + cluster.switchToActive(ns, NAMENODES[0]); + cluster.switchToStandby(ns, NAMENODES[1]); + } + } + // Start routers with only an RPC service + routerConf = new RouterConfigBuilder() + .rpc() + .build(); + + // Reduce the number of RPC clients threads to overload the Router easy + routerConf.setInt(RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY, 1); + // We decrease the DN cache times to make the test faster + routerConf.setTimeDuration( + RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); + cluster.addRouterOverrides(routerConf); + // Start routers with only an RPC service + cluster.startRouters(); + + // Register and verify all NNs with all routers + cluster.registerNamenodes(); + cluster.waitNamenodeRegistration(); + cluster.waitActiveNamespaces(); + ns0 = cluster.getNameservices().get(0); + } + + @AfterClass + public static void shutdownCluster() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Before + public void setUp() throws IOException { + router = cluster.getRandomRouter(); + routerFs = router.getFileSystem(); + routerRpcServer = router.getRouterRpcServer(); + routerRpcServer.initAsyncThreadPools(routerConf); + RouterAsyncRpcClient asyncRpcClient = new RouterAsyncRpcClient( + routerConf, router.getRouter(), routerRpcServer.getNamenodeResolver(), + routerRpcServer.getRPCMonitor(), + routerRpcServer.getRouterStateIdContext()); + RouterRpcServer spy = Mockito.spy(routerRpcServer); + Mockito.when(spy.getRPCClient()).thenReturn(asyncRpcClient); + asyncErasureCoding = new AsyncErasureCoding(spy); + + // Create mock locations + MockResolver resolver = (MockResolver) router.getRouter().getSubclusterResolver(); + resolver.addLocation("/", ns0, "/"); + FsPermission permission = new FsPermission("705"); + routerFs.mkdirs(new Path("/testdir"), permission); + FSDataOutputStream fsDataOutputStream = routerFs.create( + new Path(testfilePath), true); + fsDataOutputStream.write(new byte[1024]); + fsDataOutputStream.close(); + } + + @After + public void tearDown() throws IOException { + // clear client context + CallerContext.setCurrent(null); + boolean delete = routerFs.delete(new Path("/testdir")); + assertTrue(delete); + if (routerFs != null) { + routerFs.close(); + } + } + + @Test + public void testRouterAsyncErasureCoding() throws Exception { + String ecPolicyName = StripedFileTestUtil.getDefaultECPolicy().getName(); + HdfsFileStatus fileInfo = cluster.getNamenodes().get(0).getClient().getFileInfo(testfilePath); + assertNotNull(fileInfo); + + asyncErasureCoding.setErasureCodingPolicy("/testdir", ecPolicyName); + syncReturn(null); + + asyncErasureCoding.getErasureCodingPolicy("/testdir"); + ErasureCodingPolicy ecPolicy = syncReturn(ErasureCodingPolicy.class); + assertEquals(StripedFileTestUtil.getDefaultECPolicy().getName(), ecPolicy.getName()); + + asyncErasureCoding.getErasureCodingPolicies(); + ErasureCodingPolicyInfo[] erasureCodingPolicies = syncReturn(ErasureCodingPolicyInfo[].class); + int numECPolicies = erasureCodingPolicies.length; + ErasureCodingPolicyInfo[] erasureCodingPoliciesFromNameNode = + cluster.getNamenodes().get(0).getClient().getErasureCodingPolicies(); + + assertArrayEquals(erasureCodingPoliciesFromNameNode, erasureCodingPolicies); + + asyncErasureCoding.getErasureCodingCodecs(); + Map erasureCodingCodecs = syncReturn(Map.class); + Map erasureCodingCodecsFromNameNode = + cluster.getNamenodes().get(0).getClient().getErasureCodingCodecs(); + + assertEquals(erasureCodingCodecs, erasureCodingCodecsFromNameNode); + + // RS-12-4-1024k + final ECSchema schema = new ECSchema("rs", 12, 4); + ErasureCodingPolicy erasureCodingPolicy = new ErasureCodingPolicy(schema, 1024 * 1024); + asyncErasureCoding.addErasureCodingPolicies(new ErasureCodingPolicy[]{erasureCodingPolicy}); + AddErasureCodingPolicyResponse[] response = syncReturn(AddErasureCodingPolicyResponse[].class); + assertEquals(response[0].isSucceed(), true); + + asyncErasureCoding.getErasureCodingPolicies(); + ErasureCodingPolicyInfo[] erasureCodingPolicies2 = syncReturn(ErasureCodingPolicyInfo[].class); + int numNewECPolicies = erasureCodingPolicies2.length; + assertEquals(numECPolicies + 1, numNewECPolicies); + + asyncErasureCoding.getECTopologyResultForPolicies( + new String[]{"RS-6-3-1024k", "RS-12-4-1024k"}); + ECTopologyVerifierResult ecTResultForPolicies = syncReturn(ECTopologyVerifierResult.class); + assertEquals(false, ecTResultForPolicies.isSupported()); + + asyncErasureCoding.getECTopologyResultForPolicies( + new String[]{"XOR-2-1-1024k"}); + ECTopologyVerifierResult ecTResultForPolicies2 = syncReturn(ECTopologyVerifierResult.class); + assertEquals(true, ecTResultForPolicies2.isSupported()); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncNamenodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncNamenodeProtocol.java new file mode 100644 index 0000000000000..1814031cfb7c3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncNamenodeProtocol.java @@ -0,0 +1,126 @@ +/** + * 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.async; + +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.federation.router.RouterNamenodeProtocol; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; +import org.junit.Before; +import org.junit.Test; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Used to test the functionality of {@link RouterAsyncNamenodeProtocol}. + */ +public class TestRouterAsyncNamenodeProtocol extends RouterAsyncProtocolTestBase { + + private RouterAsyncNamenodeProtocol asyncNamenodeProtocol; + private RouterNamenodeProtocol namenodeProtocol; + + @Before + public void setup() throws Exception { + asyncNamenodeProtocol = new RouterAsyncNamenodeProtocol(getRouterAsyncRpcServer()); + namenodeProtocol = new RouterNamenodeProtocol(getRouterRpcServer()); + } + + @Test + public void getBlocks() throws Exception { + DatanodeInfo[] dns = getRouter().getClient() + .getNamenode().getDatanodeReport(HdfsConstants.DatanodeReportType.ALL); + + DatanodeInfo dn0 = dns[0]; + asyncNamenodeProtocol.getBlocks(dn0, 1024, 0, 0, + null); + BlocksWithLocations asyncRouterBlockLocations = syncReturn(BlocksWithLocations.class); + assertNotNull(asyncRouterBlockLocations); + + BlocksWithLocations syncRouterBlockLocations = namenodeProtocol.getBlocks(dn0, 1024, + 0, 0, null); + + BlockWithLocations[] asyncRouterBlocks = asyncRouterBlockLocations.getBlocks(); + BlockWithLocations[] syncRouterBlocks = syncRouterBlockLocations.getBlocks(); + + assertEquals(asyncRouterBlocks.length, syncRouterBlocks.length); + for (int i = 0; i < syncRouterBlocks.length; i++) { + assertEquals( + asyncRouterBlocks[i].getBlock().getBlockId(), + syncRouterBlocks[i].getBlock().getBlockId()); + } + } + + @Test + public void getBlockKeys() throws Exception { + asyncNamenodeProtocol.getBlockKeys(); + ExportedBlockKeys asyncBlockKeys = syncReturn(ExportedBlockKeys.class); + assertNotNull(asyncBlockKeys); + + ExportedBlockKeys syncBlockKeys = namenodeProtocol.getBlockKeys(); + compareBlockKeys(asyncBlockKeys, syncBlockKeys); + } + + @Test + public void getTransactionID() throws Exception { + asyncNamenodeProtocol.getTransactionID(); + long asyncTransactionID = syncReturn(Long.class); + assertNotNull(asyncTransactionID); + + long transactionID = namenodeProtocol.getTransactionID(); + assertEquals(asyncTransactionID, transactionID); + } + + @Test + public void getMostRecentCheckpointTxId() throws Exception { + asyncNamenodeProtocol.getMostRecentCheckpointTxId(); + long asyncMostRecentCheckpointTxId = syncReturn(Long.class); + assertNotNull(asyncMostRecentCheckpointTxId); + + long mostRecentCheckpointTxId = namenodeProtocol.getMostRecentCheckpointTxId(); + assertEquals(asyncMostRecentCheckpointTxId, mostRecentCheckpointTxId); + } + + @Test + public void versionRequest() throws Exception { + asyncNamenodeProtocol.versionRequest(); + NamespaceInfo asyncNamespaceInfo = syncReturn(NamespaceInfo.class); + assertNotNull(asyncNamespaceInfo); + NamespaceInfo syncNamespaceInfo = namenodeProtocol.versionRequest(); + compareVersion(asyncNamespaceInfo, syncNamespaceInfo); + } + + private void compareBlockKeys( + ExportedBlockKeys blockKeys, ExportedBlockKeys otherBlockKeys) { + assertEquals(blockKeys.getCurrentKey(), otherBlockKeys.getCurrentKey()); + assertEquals(blockKeys.getKeyUpdateInterval(), otherBlockKeys.getKeyUpdateInterval()); + assertEquals(blockKeys.getTokenLifetime(), otherBlockKeys.getTokenLifetime()); + } + + private void compareVersion(NamespaceInfo version, NamespaceInfo otherVersion) { + assertEquals(version.getBlockPoolID(), otherVersion.getBlockPoolID()); + assertEquals(version.getNamespaceID(), otherVersion.getNamespaceID()); + assertEquals(version.getClusterID(), otherVersion.getClusterID()); + assertEquals(version.getLayoutVersion(), otherVersion.getLayoutVersion()); + assertEquals(version.getCTime(), otherVersion.getCTime()); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncQuota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncQuota.java new file mode 100644 index 0000000000000..eecc91002ebc8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncQuota.java @@ -0,0 +1,168 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; +import org.apache.hadoop.hdfs.server.federation.MockResolver; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; +import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.ipc.CallerContext; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY; +import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES; +import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.DEFAULT_HEARTBEAT_INTERVAL_MS; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertTrue; + +public class TestRouterAsyncQuota { + private static Configuration routerConf; + /** Federated HDFS cluster. */ + private static MiniRouterDFSCluster cluster; + private static String ns0; + + /** Random Router for this federated cluster. */ + private MiniRouterDFSCluster.RouterContext router; + private FileSystem routerFs; + private RouterRpcServer routerRpcServer; + private AsyncQuota asyncQuota; + + private final String testfilePath = "/testdir/testAsyncQuota.file"; + + @BeforeClass + public static void setUpCluster() throws Exception { + cluster = new MiniRouterDFSCluster(true, 1, 2, + DEFAULT_HEARTBEAT_INTERVAL_MS, 1000); + cluster.setNumDatanodesPerNameservice(3); + cluster.setRacks( + new String[] {"/rack1", "/rack2", "/rack3"}); + cluster.startCluster(); + + // Making one Namenode active per nameservice + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + cluster.switchToActive(ns, NAMENODES[0]); + cluster.switchToStandby(ns, NAMENODES[1]); + } + } + // Start routers with only an RPC service + routerConf = new RouterConfigBuilder() + .rpc() + .quota(true) + .build(); + + // Reduce the number of RPC clients threads to overload the Router easy + routerConf.setInt(RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY, 1); + // We decrease the DN cache times to make the test faster + routerConf.setTimeDuration( + RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); + routerConf.setBoolean(DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY, true); + cluster.addRouterOverrides(routerConf); + // Start routers with only an RPC service + cluster.startRouters(); + + // Register and verify all NNs with all routers + cluster.registerNamenodes(); + cluster.waitNamenodeRegistration(); + cluster.waitActiveNamespaces(); + ns0 = cluster.getNameservices().get(0); + } + + @AfterClass + public static void shutdownCluster() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Before + public void setUp() throws IOException { + router = cluster.getRandomRouter(); + routerFs = router.getFileSystem(); + routerRpcServer = router.getRouterRpcServer(); + routerRpcServer.initAsyncThreadPools(routerConf); + RouterAsyncRpcClient asyncRpcClient = new RouterAsyncRpcClient( + routerConf, router.getRouter(), routerRpcServer.getNamenodeResolver(), + routerRpcServer.getRPCMonitor(), + routerRpcServer.getRouterStateIdContext()); + RouterRpcServer spy = Mockito.spy(routerRpcServer); + Mockito.when(spy.getRPCClient()).thenReturn(asyncRpcClient); + asyncQuota = new AsyncQuota(router.getRouter(), spy); + + // Create mock locations + MockResolver resolver = (MockResolver) router.getRouter().getSubclusterResolver(); + resolver.addLocation("/", ns0, "/"); + FsPermission permission = new FsPermission("705"); + routerFs.mkdirs(new Path("/testdir"), permission); + FSDataOutputStream fsDataOutputStream = routerFs.create( + new Path(testfilePath), true); + fsDataOutputStream.write(new byte[1024]); + fsDataOutputStream.close(); + } + + @After + public void tearDown() throws IOException { + // clear client context + CallerContext.setCurrent(null); + boolean delete = routerFs.delete(new Path("/testdir")); + assertTrue(delete); + if (routerFs != null) { + routerFs.close(); + } + } + + @Test + public void testRouterAsyncGetQuotaUsage() throws Exception { + asyncQuota.getQuotaUsage("/testdir"); + QuotaUsage quotaUsage = syncReturn(QuotaUsage.class); + // 3-replication. + Assert.assertEquals(3 * 1024, quotaUsage.getSpaceConsumed()); + // We have one directory and one file. + Assert.assertEquals(2, quotaUsage.getFileAndDirectoryCount()); + } + + @Test + public void testRouterAsyncSetQuotaUsage() throws Exception { + asyncQuota.setQuota("/testdir", Long.MAX_VALUE, 8096, StorageType.DISK, false); + syncReturn(void.class); + asyncQuota.getQuotaUsage("/testdir"); + QuotaUsage quotaUsage = syncReturn(QuotaUsage.class); + Assert.assertEquals(8096, quotaUsage.getTypeQuota(StorageType.DISK)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpc.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpc.java new file mode 100644 index 0000000000000..7290c0a0aee81 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpc.java @@ -0,0 +1,76 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; +import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; +import org.apache.hadoop.hdfs.server.federation.router.TestRouterRpc; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertArrayEquals; + +/** + * Testing the asynchronous RPC functionality of the router. + */ +public class TestRouterAsyncRpc extends TestRouterRpc { + private static MiniRouterDFSCluster cluster; + private MiniRouterDFSCluster.RouterContext rndRouter; + + @BeforeClass + public static void globalSetUp() throws Exception { + // Start routers with only an RPC service + Configuration routerConf = new RouterConfigBuilder() + .metrics() + .rpc() + .build(); + // We decrease the DN cache times to make the test faster + routerConf.setTimeDuration( + RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); + // use async router. + routerConf.setBoolean(DFS_ROUTER_ASYNC_RPC_ENABLE_KEY, true); + setUp(routerConf); + } + + @Before + public void testSetup() throws Exception { + super.testSetup(); + cluster = super.getCluster(); + // Random router for this test + rndRouter = cluster.getRandomRouter(); + } + + @Test + @Override + public void testgetGroupsForUser() throws Exception { + String[] group = new String[] {"bar", "group2"}; + UserGroupInformation.createUserForTesting("user", + new String[] {"bar", "group2"}); + rndRouter.getRouter().getRpcServer().getGroupsForUser("user"); + String[] result = syncReturn(String[].class); + assertArrayEquals(group, result); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcClient.java new file mode 100644 index 0000000000000..59c5b2de2391c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcClient.java @@ -0,0 +1,319 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; +import org.apache.hadoop.hdfs.server.federation.MockResolver; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; +import org.apache.hadoop.hdfs.server.federation.metrics.FederationRPCMetrics; +import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeContext; +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.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RemoteParam; +import org.apache.hadoop.hdfs.server.federation.router.Router; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; +import org.apache.hadoop.ipc.CallerContext; +import org.apache.hadoop.ipc.RetriableException; +import org.apache.hadoop.ipc.StandbyException; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.fs.permission.FsAction.ALL; +import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.NAMENODES; +import static org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster.DEFAULT_HEARTBEAT_INTERVAL_MS; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Used to test the functionality of {@link RouterAsyncRpcClient}. + */ +public class TestRouterAsyncRpcClient { + private static Configuration routerConf; + /** Federated HDFS cluster. */ + private static MiniRouterDFSCluster cluster; + private static String ns0; + private static String ns1; + + /** Random Router for this federated cluster. */ + private MiniRouterDFSCluster.RouterContext router; + private FileSystem routerFs; + private RouterRpcServer routerRpcServer; + private RouterAsyncRpcClient asyncRpcClient; + private FederationRPCMetrics rpcMetrics; + private final String testFile = "/test.file"; + + /** + * Start a cluster using a router service that includes 2 namespaces, + * 6 namenodes and 6 datanodes. + */ + @BeforeClass + public static void setUpCluster() throws Exception { + cluster = new MiniRouterDFSCluster(true, 2, 3, + DEFAULT_HEARTBEAT_INTERVAL_MS, 1000); + cluster.setNumDatanodesPerNameservice(3); + + cluster.startCluster(); + + // Making one Namenode active per nameservice + if (cluster.isHighAvailability()) { + for (String ns : cluster.getNameservices()) { + cluster.switchToActive(ns, NAMENODES[0]); + cluster.switchToStandby(ns, NAMENODES[1]); + cluster.switchToObserver(ns, NAMENODES[2]); + } + } + // Start routers with only an RPC service + routerConf = new RouterConfigBuilder() + .metrics() + .rpc() + .build(); + + // Reduce the number of RPC clients threads to overload the Router easy + routerConf.setInt(RBFConfigKeys.DFS_ROUTER_CLIENT_THREADS_SIZE, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_HANDLER_COUNT_KEY, 1); + routerConf.setInt(DFS_ROUTER_ASYNC_RPC_RESPONDER_COUNT_KEY, 1); + // We decrease the DN cache times to make the test faster + routerConf.setTimeDuration( + RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); + cluster.addRouterOverrides(routerConf); + // Start routers with only an RPC service + cluster.startRouters(); + + // Register and verify all NNs with all routers + cluster.registerNamenodes(); + cluster.waitNamenodeRegistration(); + cluster.waitActiveNamespaces(); + ns0 = cluster.getNameservices().get(0); + ns1 = cluster.getNameservices().get(1); + } + + @AfterClass + public static void shutdownCluster() { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Initialize the mount table, create a RouterAsyncRpcClient object, and create test file. + */ + @Before + public void setup() throws Exception { + // Create mock locations + installMockLocations(); + + router = cluster.getRandomRouter(); + rpcMetrics = router.getRouter().getRpcServer().getRPCMetrics(); + routerFs = router.getFileSystem(); + routerRpcServer = router.getRouterRpcServer(); + routerRpcServer.initAsyncThreadPools(routerConf); + + // Create a RouterAsyncRpcClient object + asyncRpcClient = new RouterAsyncRpcClient( + routerConf, router.getRouter(), routerRpcServer.getNamenodeResolver(), + routerRpcServer.getRPCMonitor(), + routerRpcServer.getRouterStateIdContext()); + + // Create a test file + FSDataOutputStream fsDataOutputStream = routerFs.create( + new Path(testFile), true); + fsDataOutputStream.write(new byte[1024]); + fsDataOutputStream.close(); + } + + @After + public void down() throws IOException { + // clear client context + CallerContext.setCurrent(null); + cluster.switchToActive(ns0, NAMENODES[0]); + asyncRpcClient.getNamenodeResolver().updateActiveNamenode( + ns0, NetUtils.createSocketAddr(cluster + .getNamenode(ns0, NAMENODES[0]).getRpcAddress())); + // Delete the test file + boolean delete = routerFs.delete(new Path(testFile)); + assertTrue(delete); + if (routerFs != null) { + routerFs.close(); + } + } + + /** + * Test the functionality of the asynchronous invokeSingle method. + */ + @Test + public void testInvokeSingle() throws Exception { + long proxyOps = rpcMetrics.getProxyOps(); + long activeProxyOps = rpcMetrics.getActiveProxyOps(); + RemoteMethod method = + new RemoteMethod(NamenodeProtocol.class, "getTransactionID"); + asyncRpcClient.invokeSingle(ns0, method); + long id = syncReturn(Long.class); + assertTrue(id > 0); + assertEquals(proxyOps + 1, rpcMetrics.getProxyOps()); + assertEquals(activeProxyOps + 1, rpcMetrics.getActiveProxyOps()); + assertTrue(rpcMetrics.getProcessingAvg() > 0); + assertTrue(rpcMetrics.getProxyAvg() > 0); + } + + /** + * Test the functionality of the asynchronous invokeAll and invokeConcurrent methods. + */ + @Test + public void testInvokeAll() throws Exception { + long proxyOps = rpcMetrics.getProxyOps(); + long activeProxyOps = rpcMetrics.getActiveProxyOps(); + final List locations = + routerRpcServer.getLocationsForPath("/multDes/dir", false); + RemoteMethod method = new RemoteMethod("mkdirs", + new Class[] {String.class, FsPermission.class, boolean.class}, + new RemoteParam(), new FsPermission(ALL, ALL, ALL), false); + asyncRpcClient.invokeAll(locations, method); + LambdaTestUtils.intercept(FileNotFoundException.class, + "Parent directory doesn't exist: /multDes", + () -> syncReturn(boolean.class)); + assertEquals(proxyOps + 2, rpcMetrics.getProxyOps()); + assertEquals(activeProxyOps + 2, rpcMetrics.getActiveProxyOps()); + + proxyOps = rpcMetrics.getProxyOps(); + activeProxyOps = rpcMetrics.getActiveProxyOps(); + method = new RemoteMethod("mkdirs", + new Class[] {String.class, FsPermission.class, boolean.class}, + new RemoteParam(), new FsPermission(ALL, ALL, ALL), true); + asyncRpcClient.invokeAll(locations, method); + Boolean success = syncReturn(Boolean.class); + assertTrue(success); + assertEquals(proxyOps + 2, rpcMetrics.getProxyOps()); + assertEquals(activeProxyOps + 2, rpcMetrics.getActiveProxyOps()); + + FileStatus[] fileStatuses = routerFs.listStatus(new Path("/multDes")); + assertNotNull(fileStatuses); + assertTrue(rpcMetrics.getProcessingAvg() > 0); + assertTrue(rpcMetrics.getProxyAvg() > 0); + } + + /** + * Test the functionality of the asynchronous invokeMethod method. + */ + @Test + public void testInvokeMethod() throws Exception { + long proxyOps = rpcMetrics.getProxyOps(); + long activeProxyOps = rpcMetrics.getActiveProxyOps(); + RemoteMethod method = new RemoteMethod("getFileInfo", + new Class[] {String.class}, new RemoteParam()); + UserGroupInformation ugi = RouterRpcServer.getRemoteUser(); + Class protocol = method.getProtocol(); + Object[] params = new String[]{testFile}; + List namenodes = + asyncRpcClient.getOrderedNamenodes(ns0, false); + asyncRpcClient.invokeMethod(ugi, namenodes, false, + protocol, method.getMethod(), params); + FileStatus fileStatus = syncReturn(FileStatus.class); + assertEquals(1024, fileStatus.getLen()); + assertEquals(proxyOps + 1, rpcMetrics.getProxyOps()); + assertEquals(activeProxyOps + 1, rpcMetrics.getActiveProxyOps()); + + LambdaTestUtils.intercept(IOException.class, + "No namenodes to invoke", + () -> asyncRpcClient.invokeMethod(ugi, new ArrayList<>(), false, + protocol, method.getMethod(), params)); + + proxyOps = rpcMetrics.getProxyOps(); + activeProxyOps = rpcMetrics.getActiveProxyOps(); + asyncRpcClient.invokeMethod(ugi, namenodes.subList(1, 3), false, + protocol, method.getMethod(), params); + LambdaTestUtils.intercept(StandbyException.class, + "No namenode available to invoke getFileInfo", + () -> syncReturn(FileStatus.class)); + assertEquals(proxyOps, rpcMetrics.getProxyOps()); + assertEquals(activeProxyOps, rpcMetrics.getActiveProxyOps()); + + cluster.switchToStandby(ns0, NAMENODES[0]); + asyncRpcClient.getNamenodeResolver().updateUnavailableNamenode( + ns0, NetUtils.createSocketAddr(namenodes.get(0).getRpcAddress())); + asyncRpcClient.invokeMethod(ugi, namenodes, false, + protocol, method.getMethod(), params); + LambdaTestUtils.intercept(RetriableException.class, + "No namenodes available under nameservice ns0", + () -> syncReturn(FileStatus.class)); + assertEquals(1, rpcMetrics.getProxyOpNoNamenodes()); + + asyncRpcClient.invokeMethod(ugi, namenodes, false, + null, method.getMethod(), params); + LambdaTestUtils.intercept(StandbyException.class, + "Cannot get a connection", + () -> syncReturn(FileStatus.class)); + assertEquals(1, rpcMetrics.getProxyOpFailureCommunicate()); + } + + /** + * Test the functionality of the asynchronous invokeSequential method. + */ + @Test + public void testInvokeSequential() throws Exception { + List locations = + routerRpcServer.getLocationsForPath(testFile, false, false); + RemoteMethod remoteMethod = new RemoteMethod("getBlockLocations", + new Class[] {String.class, long.class, long.class}, + new RemoteParam(), 0, 1024); + asyncRpcClient.invokeSequential(locations, remoteMethod, + LocatedBlocks.class, null); + LocatedBlocks locatedBlocks = syncReturn(LocatedBlocks.class); + assertEquals(1024, locatedBlocks.getFileLength()); + assertEquals(1, locatedBlocks.getLocatedBlocks().size()); + } + + /** + * Initialize the mount information. + */ + private void installMockLocations() { + List routers = cluster.getRouters(); + + for (MiniRouterDFSCluster.RouterContext rc : routers) { + Router r = rc.getRouter(); + MockResolver resolver = (MockResolver) r.getSubclusterResolver(); + resolver.addLocation("/", ns0, "/"); + resolver.addLocation("/multDes", ns0, "/multDes"); + resolver.addLocation("/multDes", ns1, "/multDes"); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcMultiDestination.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcMultiDestination.java new file mode 100644 index 0000000000000..ec1ff0ce97b9f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcMultiDestination.java @@ -0,0 +1,66 @@ +/** + * 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.async; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; +import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; +import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys; +import org.apache.hadoop.hdfs.server.federation.router.TestRouterRpcMultiDestination; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_ASYNC_RPC_ENABLE_KEY; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertArrayEquals; + +/** + * Testing the asynchronous RPC functionality of the router with multiple mounts. + */ +public class TestRouterAsyncRpcMultiDestination extends TestRouterRpcMultiDestination { + + @BeforeClass + public static void globalSetUp() throws Exception { + // Start routers with only an RPC service + Configuration routerConf = new RouterConfigBuilder() + .metrics() + .rpc() + .build(); + // We decrease the DN cache times to make the test faster + routerConf.setTimeDuration( + RBFConfigKeys.DN_REPORT_CACHE_EXPIRE, 1, TimeUnit.SECONDS); + // use async router. + routerConf.setBoolean(DFS_ROUTER_ASYNC_RPC_ENABLE_KEY, true); + setUp(routerConf); + } + + @Test + @Override + public void testgetGroupsForUser() throws Exception { + MiniRouterDFSCluster.RouterContext rndRouter = super.getRouterContext(); + String[] group = new String[] {"bar", "group2"}; + UserGroupInformation.createUserForTesting("user", + new String[] {"bar", "group2"}); + rndRouter.getRouter().getRpcServer().getGroupsForUser("user"); + String[] result = syncReturn(String[].class); + assertArrayEquals(group, result); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcServer.java new file mode 100644 index 0000000000000..c022789a2eba8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncRpcServer.java @@ -0,0 +1,96 @@ +/** + * 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.async; + +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation; +import org.apache.hadoop.hdfs.server.federation.router.RemoteMethod; +import org.apache.hadoop.hdfs.server.federation.router.RouterRpcServer; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +/** + * Used to test the async functionality of {@link RouterRpcServer}. + */ +public class TestRouterAsyncRpcServer extends RouterAsyncProtocolTestBase { + private RouterRpcServer asyncRouterRpcServer; + + @Before + public void setup() throws IOException { + asyncRouterRpcServer = getRouterAsyncRpcServer(); + } + + /** + * Test that the async RPC server can invoke a method at an available Namenode. + */ + @Test + public void testInvokeAtAvailableNsAsync() throws Exception { + RemoteMethod method = new RemoteMethod("getStoragePolicies"); + asyncRouterRpcServer.invokeAtAvailableNsAsync(method, BlockStoragePolicy[].class); + BlockStoragePolicy[] storagePolicies = syncReturn(BlockStoragePolicy[].class); + assertEquals(8, storagePolicies.length); + } + + /** + * Test get create location async. + */ + @Test + public void testGetCreateLocationAsync() throws Exception { + final List locations = + asyncRouterRpcServer.getLocationsForPath("/testdir", true); + asyncRouterRpcServer.getCreateLocationAsync("/testdir", locations); + RemoteLocation remoteLocation = syncReturn(RemoteLocation.class); + assertNotNull(remoteLocation); + assertEquals(getNs0(), remoteLocation.getNameserviceId()); + } + + /** + * Test get datanode report async. + */ + @Test + public void testGetDatanodeReportAsync() throws Exception { + asyncRouterRpcServer.getDatanodeReportAsync( + HdfsConstants.DatanodeReportType.ALL, true, 0); + DatanodeInfo[] datanodeInfos = syncReturn(DatanodeInfo[].class); + assertEquals(3, datanodeInfos.length); + + // Get the namespace where the datanode is located + asyncRouterRpcServer.getDatanodeStorageReportMapAsync(HdfsConstants.DatanodeReportType.ALL); + Map map = syncReturn(Map.class); + assertEquals(1, map.size()); + assertEquals(3, map.get(getNs0()).length); + + DatanodeInfo[] slowDatanodeReport1 = + asyncRouterRpcServer.getSlowDatanodeReport(true, 0); + + asyncRouterRpcServer.getSlowDatanodeReportAsync(true, 0); + DatanodeInfo[] slowDatanodeReport2 = syncReturn(DatanodeInfo[].class); + assertEquals(slowDatanodeReport1, slowDatanodeReport2); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncSnapshot.java new file mode 100644 index 0000000000000..a44664ec23dd7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncSnapshot.java @@ -0,0 +1,115 @@ +/** + * 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.async; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing; +import org.apache.hadoop.hdfs.protocol.SnapshotException; +import org.apache.hadoop.hdfs.protocol.SnapshotStatus; +import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; +import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.Before; +import org.junit.Test; +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType.MODIFY; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertEquals; + +/** + * Used to test the functionality of {@link RouterAsyncSnapshot}. + */ +public class TestRouterAsyncSnapshot extends RouterAsyncProtocolTestBase { + private final String testFile = "/testdir/testSnapshot.file"; + private FileSystem routerFs; + private RouterAsyncSnapshot asyncSnapshot; + + @Before + public void setup() throws IOException { + routerFs = getRouterFs(); + asyncSnapshot = new RouterAsyncSnapshot(getRouterAsyncRpcServer()); + FSDataOutputStream fsDataOutputStream = routerFs.create( + new Path(testFile), true); + fsDataOutputStream.write(new byte[1024]); + fsDataOutputStream.close(); + } + + @Test + public void testRouterAsyncSnapshot() throws Exception { + asyncSnapshot.allowSnapshot("/testdir"); + syncReturn(null); + asyncSnapshot.createSnapshot("/testdir", "testdirSnapshot"); + String snapshotName = syncReturn(String.class); + assertEquals("/testdir/.snapshot/testdirSnapshot", snapshotName); + asyncSnapshot.getSnapshottableDirListing(); + SnapshottableDirectoryStatus[] snapshottableDirectoryStatuses = + syncReturn(SnapshottableDirectoryStatus[].class); + assertEquals(1, snapshottableDirectoryStatuses.length); + asyncSnapshot.getSnapshotListing("/testdir"); + SnapshotStatus[] snapshotStatuses = syncReturn(SnapshotStatus[].class); + assertEquals(1, snapshotStatuses.length); + + FSDataOutputStream fsDataOutputStream = routerFs.append( + new Path("/testdir/testSnapshot.file"), true); + fsDataOutputStream.write(new byte[1024]); + fsDataOutputStream.close(); + + asyncSnapshot.createSnapshot("/testdir", "testdirSnapshot1"); + snapshotName = syncReturn(String.class); + assertEquals("/testdir/.snapshot/testdirSnapshot1", snapshotName); + + asyncSnapshot.getSnapshotDiffReport("/testdir", + "testdirSnapshot", "testdirSnapshot1"); + SnapshotDiffReport snapshotDiffReport = syncReturn(SnapshotDiffReport.class); + assertEquals(MODIFY, snapshotDiffReport.getDiffList().get(0).getType()); + + asyncSnapshot.getSnapshotDiffReportListing("/testdir", + "testdirSnapshot", "testdirSnapshot1", new byte[]{}, -1); + SnapshotDiffReportListing snapshotDiffReportListing = + syncReturn(SnapshotDiffReportListing.class); + assertEquals(1, snapshotDiffReportListing.getModifyList().size()); + + LambdaTestUtils.intercept(SnapshotException.class, () -> { + asyncSnapshot.disallowSnapshot("/testdir"); + syncReturn(null); + }); + + asyncSnapshot.renameSnapshot("/testdir", + "testdirSnapshot1", "testdirSnapshot2"); + syncReturn(null); + + LambdaTestUtils.intercept(SnapshotException.class, + "Cannot delete snapshot testdirSnapshot1 from path /testdir", + () -> { + asyncSnapshot.deleteSnapshot("/testdir", "testdirSnapshot1"); + syncReturn(null); + }); + + asyncSnapshot.deleteSnapshot("/testdir", "testdirSnapshot2"); + syncReturn(null); + + asyncSnapshot.deleteSnapshot("/testdir", "testdirSnapshot"); + syncReturn(null); + + asyncSnapshot.disallowSnapshot("/testdir"); + syncReturn(null); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncStoragePolicy.java new file mode 100644 index 0000000000000..d2afe9ad4af05 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncStoragePolicy.java @@ -0,0 +1,66 @@ +/** + * 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.async; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.junit.Before; +import org.junit.Test; +import java.io.IOException; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +/** + * Used to test the functionality of {@link RouterAsyncStoragePolicy}. + */ +public class TestRouterAsyncStoragePolicy extends RouterAsyncProtocolTestBase { + private final String testfilePath = "/testdir/testAsyncStoragePolicy.file"; + private RouterAsyncStoragePolicy asyncStoragePolicy; + + @Before + public void setup() throws IOException { + asyncStoragePolicy = new RouterAsyncStoragePolicy(getRouterAsyncRpcServer()); + FSDataOutputStream fsDataOutputStream = getRouterFs().create( + new Path(testfilePath), true); + fsDataOutputStream.write(new byte[1024]); + fsDataOutputStream.close(); + } + + @Test + public void testRouterAsyncStoragePolicy() throws Exception { + BlockStoragePolicy[] storagePolicies = getCluster().getNamenodes().get(0) + .getClient().getStoragePolicies(); + asyncStoragePolicy.getStoragePolicies(); + BlockStoragePolicy[] storagePoliciesAsync = syncReturn(BlockStoragePolicy[].class); + assertArrayEquals(storagePolicies, storagePoliciesAsync); + + asyncStoragePolicy.getStoragePolicy(testfilePath); + BlockStoragePolicy blockStoragePolicy1 = syncReturn(BlockStoragePolicy.class); + + asyncStoragePolicy.setStoragePolicy(testfilePath, "COLD"); + syncReturn(null); + asyncStoragePolicy.getStoragePolicy(testfilePath); + BlockStoragePolicy blockStoragePolicy2 = syncReturn(BlockStoragePolicy.class); + assertNotEquals(blockStoragePolicy1, blockStoragePolicy2); + assertEquals("COLD", blockStoragePolicy2.getName()); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncUserProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncUserProtocol.java new file mode 100644 index 0000000000000..ce76be9ed7bf0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncUserProtocol.java @@ -0,0 +1,48 @@ +/** + * 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.async; + +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Before; +import org.junit.Test; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.syncReturn; +import static org.junit.Assert.assertArrayEquals; + +/** + * Used to test the functionality of {@link RouterAsyncUserProtocol}. + */ +public class TestRouterAsyncUserProtocol extends RouterAsyncProtocolTestBase { + + private RouterAsyncUserProtocol asyncUserProtocol; + + @Before + public void setup() throws Exception { + asyncUserProtocol = new RouterAsyncUserProtocol(getRouterAsyncRpcServer()); + } + + @Test + public void testgetGroupsForUser() throws Exception { + String[] group = new String[] {"bar", "group2"}; + UserGroupInformation.createUserForTesting("user", + new String[] {"bar", "group2"}); + asyncUserProtocol.getGroupsForUser("user"); + String[] result = syncReturn(String[].class); + assertArrayEquals(group, result); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncClass.java new file mode 100644 index 0000000000000..bfc172edc02c8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/AsyncClass.java @@ -0,0 +1,249 @@ +/** + * 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.async.utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.function.Function; + +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCatch; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncCurrent; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncFinally; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncForEach; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncThrowException; +import static org.apache.hadoop.hdfs.server.federation.router.async.utils.AsyncUtil.asyncTry; + +/** + * AsyncClass demonstrates the conversion of synchronous methods + * from SyncClass into asynchronous operations using AsyncUtil. + * This class overrides methods with asynchronous logic, enhancing + * the performance by allowing non-blocking task execution. + * + *

+ * By utilizing AsyncUtil's utility methods, such as asyncApply, + * asyncForEach, and others, each method in AsyncClass can perform + * time-consuming tasks on a separate thread, thus not blocking + * the main execution thread. + *

+ * + *

+ * For example, the applyMethod in AsyncClass is an async version of + * the same method in SyncClass. It uses asyncApply to schedule + * the timeConsumingMethod to run asynchronously and returns a + * CompletableFuture that will be completed with the result of + * the operation. + *

+ * + *

+ * This class serves as an example of how to transform synchronous + * operations into asynchronous ones using the AsyncUtil tools, + * which can be applied to other parts of the HDFS Federation + * router or similar systems to improve concurrency and + * performance. + *

+ * + * @see SyncClass + * @see AsyncUtil + * @see CompletableFuture + */ +public class AsyncClass extends SyncClass{ + private static final Logger LOG = + LoggerFactory.getLogger(AsyncClass.class); + private ExecutorService executorService; + private final static String ASYNC_WORKER = "Async Worker"; + + public AsyncClass(long timeConsuming) { + super(timeConsuming); + executorService = Executors.newFixedThreadPool(1, r -> { + Thread asyncWork = new Thread(r); + asyncWork.setDaemon(true); + asyncWork.setName(ASYNC_WORKER); + return asyncWork; + }); + } + + @Override + public String applyMethod(int input) { + timeConsumingMethod(input); + asyncApply(res -> { + return "applyMethod" + res; + }); + return asyncReturn(String.class); + } + + @Override + public String applyMethod(int input, boolean canException) { + timeConsumingMethod(input); + asyncApply(res -> { + if (canException) { + if (res.equals("[2]")) { + throw new IOException("input 2 exception"); + } else if (res.equals("[3]")) { + throw new RuntimeException("input 3 exception"); + } + } + return res; + }); + return asyncReturn(String.class); + } + + @Override + public String exceptionMethod(int input) { + if (input == 2) { + asyncThrowException(new IOException("input 2 exception")); + return null; + } else if (input == 3) { + asyncThrowException(new RuntimeException("input 3 exception")); + return null; + } + return applyMethod(input); + } + + @Override + public String forEachMethod(List list) { + StringBuilder result = new StringBuilder(); + asyncForEach(list.iterator(), + (forEach, input) -> { + timeConsumingMethod(input); + asyncApply(res -> { + result.append("forEach" + res + ","); + return result.toString(); + }); + }); + return asyncReturn(String.class); + } + + @Override + public String forEachBreakMethod(List list) { + StringBuilder result = new StringBuilder(); + asyncForEach(list.iterator(), + (forEach, input) -> { + timeConsumingMethod(input); + asyncApply(res -> { + if (res.equals("[2]")) { + forEach.breakNow(); + } else { + result.append("forEach" + res + ","); + } + return result.toString(); + }); + }); + return asyncReturn(String.class); + } + + @Override + public String forEachBreakByExceptionMethod(List list) { + StringBuilder result = new StringBuilder(); + asyncForEach(list.iterator(), + (forEach, input) -> { + asyncTry(() -> { + applyMethod(input, true); + asyncApply(res -> { + result.append("forEach" + res + ","); + return result.toString(); + }); + }); + asyncCatch((res, e) -> { + if (e instanceof IOException) { + result.append(e + ","); + } else if (e instanceof RuntimeException) { + forEach.breakNow(); + } + return result.toString(); + }, Exception.class); + }); + return asyncReturn(String.class); + } + + @Override + public String applyThenApplyMethod(int input) { + timeConsumingMethod(input); + asyncApply((AsyncApplyFunction) res -> { + if (res.equals("[1]")) { + timeConsumingMethod(2); + } else { + asyncComplete(res); + } + }); + return asyncReturn(String.class); + } + + @Override + public String applyCatchThenApplyMethod(int input) { + asyncTry(() -> applyMethod(input, true)); + asyncCatch((AsyncCatchFunction) (res, ioe) -> { + applyMethod(1); + }, IOException.class); + return asyncReturn(String.class); + } + + @Override + public String applyCatchFinallyMethod( + int input, List resource) { + asyncTry(() -> applyMethod(input, true)); + asyncCatch((res, e) -> { + throw new IOException("Catch " + e.getMessage()); + }, IOException.class); + asyncFinally((FinallyFunction) res -> { + resource.clear(); + return res; + }); + return asyncReturn(String.class); + } + + @Override + public String currentMethod(List list) { + asyncCurrent(list, + input -> applyMethod(input, true), + (Function[], String>) futures -> { + StringBuilder result = new StringBuilder(); + for (Future future : futures) { + try { + String res = future.get(); + result.append(res + ","); + } catch (Exception e) { + result.append(e.getMessage() + ","); + } + } + return result.toString(); + }); + return asyncReturn(String.class); + } + + @Override + public String timeConsumingMethod(int input) { + CompletableFuture result = CompletableFuture + .supplyAsync(() -> { + LOG.info("[{} thread] invoke consumingMethod for parameter: {}", + Thread.currentThread().getName(), input); + return AsyncClass.super.timeConsumingMethod(input); + }, executorService); + Async.CUR_COMPLETABLE_FUTURE.set(result); + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/BaseClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/BaseClass.java new file mode 100644 index 0000000000000..084806d65c1af --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/BaseClass.java @@ -0,0 +1,66 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.List; + +/** + * It defines a set of methods that can be executed either synchronously + * or asynchronously, depending on the implementation. + * + *

+ * This interface is designed to abstract the common operations that need + * to be performed in a time-consuming manner, such as processing a list + * of items or applying a method that involves I/O operations. By defining + * these methods in an interface, it allows for both synchronous and + * asynchronous implementations, providing flexibility and the ability to + * improve performance without changing the external API. + *

+ * + *

+ * Implementations of this interface are expected to provide concrete + * implementations of the defined methods, either by performing the + * operations synchronously in a blocking manner or by performing them + * asynchronously in a non-blocking manner. + *

+ * + * @see SyncClass + * @see AsyncClass + */ +public interface BaseClass { + String applyMethod(int input); + + String applyMethod(int input, boolean canException) throws IOException; + + String exceptionMethod(int input) throws IOException; + + String forEachMethod(List list); + + String forEachBreakMethod(List list); + + String forEachBreakByExceptionMethod(List list); + + String applyThenApplyMethod(int input); + + String applyCatchThenApplyMethod(int input); + + String applyCatchFinallyMethod(int input, List resource) throws IOException; + + String currentMethod(List list); +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/SyncClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/SyncClass.java new file mode 100644 index 0000000000000..805b955661d5c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/SyncClass.java @@ -0,0 +1,194 @@ +/** + * 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.async.utils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +/** + * SyncClass implements BaseClass, providing a synchronous + * version of the methods. All operations are performed in a + * blocking manner, waiting for completion before proceeding. + * + * This class is the foundation for the AsyncClass, which + * provides asynchronous implementations. + * + * @see BaseClass + * @see AsyncClass + */ +public class SyncClass implements BaseClass{ + private long timeConsuming; + + public SyncClass(long timeConsuming) { + this.timeConsuming = timeConsuming; + } + + @Override + public String applyMethod(int input) { + String res = timeConsumingMethod(input); + return "applyMethod" + res; + } + + @Override + public String applyMethod(int input, boolean canException) throws IOException { + String res = timeConsumingMethod(input); + if (canException) { + if (res.equals("[2]")) { + throw new IOException("input 2 exception"); + } else if (res.equals("[3]")) { + throw new RuntimeException("input 3 exception"); + } + } + return res; + } + + @Override + public String exceptionMethod(int input) throws IOException { + if (input == 2) { + throw new IOException("input 2 exception"); + } else if (input == 3) { + throw new RuntimeException("input 3 exception"); + } + return applyMethod(input); + } + + @Override + public String forEachMethod(List list) { + StringBuilder result = new StringBuilder(); + for (int input : list) { + String res = timeConsumingMethod(input); + result.append("forEach" + res + ","); + } + return result.toString(); + } + + @Override + public String forEachBreakMethod(List list) { + StringBuilder result = new StringBuilder(); + for (int input : list) { + String res = timeConsumingMethod(input); + if (res.equals("[2]")) { + break; + } + result.append("forEach" + res + ","); + } + return result.toString(); + } + + @Override + public String forEachBreakByExceptionMethod(List list) { + StringBuilder result = new StringBuilder(); + for (int input : list) { + try { + String res = applyMethod(input, true); + result.append("forEach" + res + ","); + } catch (IOException e) { + result.append(e + ","); + } catch (RuntimeException e) { + break; + } + } + return result.toString(); + } + + @Override + public String applyThenApplyMethod(int input) { + String res = timeConsumingMethod(input); + if (res.equals("[1]")) { + res = timeConsumingMethod(2); + } + return res; + } + + @Override + public String applyCatchThenApplyMethod(int input) { + String res = null; + try { + res = applyMethod(input, true); + } catch (IOException e) { + res = applyMethod(1); + } + return res; + } + + @Override + public String applyCatchFinallyMethod( + int input, List resource) throws IOException { + String res = null; + try { + res = applyMethod(input, true); + } catch (IOException e) { + throw new IOException("Catch " + e.getMessage()); + } finally { + resource.clear(); + } + return res; + } + + @Override + public String currentMethod(List list) { + ExecutorService executor = getExecutorService(); + List> futures = new ArrayList<>(); + for (int input : list) { + Future future = executor.submit( + () -> applyMethod(input, true)); + futures.add(future); + } + + StringBuilder result = new StringBuilder(); + for (Future future : futures) { + try { + String res = future.get(); + result.append(res + ","); + } catch (Exception e) { + result.append(e.getMessage() + ","); + } + } + return result.toString(); + } + + + /** + * Simulates a synchronous method that performs + * a time-consuming task and returns a result. + * + * @param input The input parameter for the method. + * @return A string that represents the result of the method. + */ + public String timeConsumingMethod(int input) { + try { + Thread.sleep(timeConsuming); + return "[" + input + "]"; + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return "Error:" + e.getMessage(); + } + } + + private ExecutorService getExecutorService() { + return Executors.newFixedThreadPool(2, r -> { + Thread t = new Thread(r); + t.setDaemon(true); + return t; + }); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/TestAsyncUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/TestAsyncUtil.java new file mode 100644 index 0000000000000..644f639ac9e5a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/utils/TestAsyncUtil.java @@ -0,0 +1,277 @@ +/** + * 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.async.utils; + +import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.util.Time; +import org.junit.After; +import org.junit.Before; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Callable; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * The TestAsyncUtil class provides a suite of test cases for the + * asynchronous utility class AsyncUtil. It utilizes the JUnit testing + * framework to verify that asynchronous operations are performed as + * expected. + * + *

+ * This class contains multiple test methods designed to test various + * asynchronous operation scenarios, including: + *

    + *
  • testApply - Tests the asynchronous application of a method.
  • + *
  • testApplyException - Tests exception handling in + * asynchronous methods.
  • + *
  • testApplyThenApplyMethod - Tests the chaining of + * asynchronous method calls.
  • + *
  • testCatchThenApplyMethod - Tests the invocation of + * asynchronous methods after exception catching.
  • + *
  • testForEach - Tests asynchronous iteration operations.
  • + *
  • testForEachBreak - Tests asynchronous iteration with break + * conditions.
  • + *
  • testForEachBreakByException - Tests the interruption of + * asynchronous iteration due to exceptions.
  • + *
+ *

+ * + * The tests cover both synchronous (Sync) and asynchronous (Async) + * configurations to ensure consistent behavior under different + * execution modes. + * + * @see AsyncUtil + * @see BaseClass + * @see SyncClass + * @see AsyncClass + */ +public class TestAsyncUtil { + private static final Logger LOG = + LoggerFactory.getLogger(TestAsyncUtil.class); + private static final long TIME_CONSUMING = 100; + private BaseClass baseClass; + private boolean enableAsync; + + public enum ExecutionMode { + SYNC, + ASYNC + } + + @Before + public void setUp(ExecutionMode mode) { + if (mode.equals(ExecutionMode.ASYNC)) { + baseClass = new AsyncClass(TIME_CONSUMING); + enableAsync = true; + } else { + baseClass = new SyncClass(TIME_CONSUMING); + } + } + + @After + public void after() { + baseClass = null; + enableAsync = false; + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testApply(ExecutionMode mode) + throws Exception { + setUp(mode); + long start = Time.monotonicNow(); + String result = baseClass.applyMethod(1); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", mode, cost); + checkResult("applyMethod[1]", result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testApplyException(ExecutionMode mode) throws Exception { + setUp(mode); + checkException( + () -> baseClass.applyMethod(2, true), + IOException.class, "input 2 exception"); + + checkException( + () -> baseClass.applyMethod(3, true), + RuntimeException.class, "input 3 exception"); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testExceptionMethod(ExecutionMode mode) throws Exception { + setUp(mode); + checkException( + () -> baseClass.exceptionMethod(2), + IOException.class, "input 2 exception"); + + checkException( + () -> baseClass.exceptionMethod(3), + RuntimeException.class, "input 3 exception"); + + long start = Time.monotonicNow(); + String result = baseClass.exceptionMethod(1); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", mode, cost); + checkResult("applyMethod[1]", result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testApplyThenApplyMethod(ExecutionMode mode) throws Exception { + setUp(mode); + long start = Time.monotonicNow(); + String result = baseClass.applyThenApplyMethod(1); + long cost = Time.monotonicNow() - start; + checkResult("[2]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", mode, cost); + + start = Time.monotonicNow(); + result = baseClass.applyThenApplyMethod(3); + cost = Time.monotonicNow() - start; + checkResult("[3]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", mode, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCatchThenApplyMethod(ExecutionMode mode) throws Exception { + setUp(mode); + long start = Time.monotonicNow(); + String result = baseClass.applyCatchThenApplyMethod(2); + long cost = Time.monotonicNow() - start; + checkResult("applyMethod[1]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", mode, cost); + + start = Time.monotonicNow(); + result = baseClass.applyCatchThenApplyMethod(0); + cost = Time.monotonicNow() - start; + checkResult("[0]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", mode, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCatchFinallyMethod(ExecutionMode mode) throws Exception { + setUp(mode); + List resource = new ArrayList<>(); + resource.add("resource1"); + checkException( + () -> baseClass.applyCatchFinallyMethod(2, resource), + IOException.class, "input 2 exception"); + assertTrue(resource.size() == 0); + + long start = Time.monotonicNow(); + String result = baseClass.applyCatchFinallyMethod(0, resource); + long cost = Time.monotonicNow() - start; + checkResult("[0]", result, TIME_CONSUMING, cost); + assertTrue(resource.size() == 0); + LOG.info("[{}] main thread cost: {} ms", mode, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEach(ExecutionMode mode) throws Exception { + setUp(mode); + long start = Time.monotonicNow(); + String result = baseClass.forEachMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", mode, cost); + checkResult("forEach[1],forEach[2],forEach[3],", result, + TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEachBreak(ExecutionMode mode) throws Exception { + setUp(mode); + long start = Time.monotonicNow(); + String result = baseClass.forEachBreakMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", mode, cost); + checkResult("forEach[1],", result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEachBreakByException(ExecutionMode mode) + throws Exception { + setUp(mode); + long start = Time.monotonicNow(); + String result = baseClass.forEachBreakByExceptionMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", mode, cost); + checkResult("forEach[1],java.io.IOException: input 2 exception,", + result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCurrentMethod(ExecutionMode mode) + throws Exception { + setUp(mode); + long start = Time.monotonicNow(); + String result = baseClass.currentMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", mode, cost); + checkResult("[1],java.io.IOException: input 2 exception," + + "java.lang.RuntimeException: input 3 exception,", + result, TIME_CONSUMING, cost); + } + + private void checkResult( + String result, String actualResult, long cost, long actualCost) + throws Exception { + if (enableAsync) { + Assertions.assertNull(actualResult); + actualResult = AsyncUtil.syncReturn(String.class); + assertNotNull(actualResult); + assertTrue(actualCost < cost); + } else { + assertFalse(actualCost < cost); + } + assertEquals(result, actualResult); + } + + private < E extends Throwable> void checkException( + Callable eval, Class clazz, String contained) throws Exception { + if (enableAsync) { + LambdaTestUtils.intercept(clazz, contained, + () -> { + eval.call(); + return AsyncUtil.syncReturn(String.class); + }); + } else { + LambdaTestUtils.intercept(clazz, contained, () -> { + String res = eval.call(); + return res; + }); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/TestRouterSecurityManager.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/TestRouterSecurityManager.java index 9b2b5a0658877..b73764cbe18a8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/TestRouterSecurityManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/TestRouterSecurityManager.java @@ -132,9 +132,7 @@ public void testDelegationTokens() throws IOException { // Cancel the delegation token securityManager.cancelDelegationToken(token); - String exceptionCause = "Renewal request for unknown token"; exceptionRule.expect(SecretManager.InvalidToken.class); - exceptionRule.expectMessage(exceptionCause); // This throws an exception as token has been cancelled. securityManager.renewDelegationToken(token); 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..47629d87a58ea 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 @@ -341,117 +341,109 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements VOID_UNSET_STORAGE_POLICY_RESPONSE = UnsetStoragePolicyResponseProto.newBuilder().build(); - private static final CreateResponseProto VOID_CREATE_RESPONSE = - CreateResponseProto.newBuilder().build(); + protected static final CreateResponseProto VOID_CREATE_RESPONSE = + CreateResponseProto.newBuilder().build(); - private static final SetPermissionResponseProto VOID_SET_PERM_RESPONSE = - SetPermissionResponseProto.newBuilder().build(); + protected static final SetPermissionResponseProto VOID_SET_PERM_RESPONSE = + SetPermissionResponseProto.newBuilder().build(); - private static final SetOwnerResponseProto VOID_SET_OWNER_RESPONSE = - SetOwnerResponseProto.newBuilder().build(); + protected static final SetOwnerResponseProto VOID_SET_OWNER_RESPONSE = + SetOwnerResponseProto.newBuilder().build(); - private static final AbandonBlockResponseProto VOID_ADD_BLOCK_RESPONSE = - AbandonBlockResponseProto.newBuilder().build(); + protected static final AbandonBlockResponseProto VOID_ADD_BLOCK_RESPONSE = + AbandonBlockResponseProto.newBuilder().build(); - private static final ReportBadBlocksResponseProto VOID_REP_BAD_BLOCK_RESPONSE = - ReportBadBlocksResponseProto.newBuilder().build(); + protected static final ReportBadBlocksResponseProto VOID_REP_BAD_BLOCK_RESPONSE = + ReportBadBlocksResponseProto.newBuilder().build(); - private static final ConcatResponseProto VOID_CONCAT_RESPONSE = - ConcatResponseProto.newBuilder().build(); + protected static final ConcatResponseProto VOID_CONCAT_RESPONSE = + ConcatResponseProto.newBuilder().build(); - private static final Rename2ResponseProto VOID_RENAME2_RESPONSE = - Rename2ResponseProto.newBuilder().build(); + protected static final Rename2ResponseProto VOID_RENAME2_RESPONSE = + Rename2ResponseProto.newBuilder().build(); - private static final GetListingResponseProto VOID_GETLISTING_RESPONSE = - GetListingResponseProto.newBuilder().build(); + 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 = - RenewLeaseResponseProto.newBuilder().build(); + protected static final RenewLeaseResponseProto VOID_RENEWLEASE_RESPONSE = + RenewLeaseResponseProto.newBuilder().build(); - private static final RefreshNodesResponseProto VOID_REFRESHNODES_RESPONSE = - RefreshNodesResponseProto.newBuilder().build(); + protected static final RefreshNodesResponseProto VOID_REFRESHNODES_RESPONSE = + RefreshNodesResponseProto.newBuilder().build(); - private static final FinalizeUpgradeResponseProto VOID_FINALIZEUPGRADE_RESPONSE = - FinalizeUpgradeResponseProto.newBuilder().build(); + protected static final FinalizeUpgradeResponseProto VOID_FINALIZEUPGRADE_RESPONSE = + FinalizeUpgradeResponseProto.newBuilder().build(); - private static final MetaSaveResponseProto VOID_METASAVE_RESPONSE = - MetaSaveResponseProto.newBuilder().build(); + protected static final MetaSaveResponseProto VOID_METASAVE_RESPONSE = + MetaSaveResponseProto.newBuilder().build(); - private static final GetFileInfoResponseProto VOID_GETFILEINFO_RESPONSE = - GetFileInfoResponseProto.newBuilder().build(); + 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 = - GetFileLinkInfoResponseProto.newBuilder().build(); + protected static final GetFileLinkInfoResponseProto VOID_GETFILELINKINFO_RESPONSE = + GetFileLinkInfoResponseProto.newBuilder().build(); - private static final SetQuotaResponseProto VOID_SETQUOTA_RESPONSE = - SetQuotaResponseProto.newBuilder().build(); + protected static final SetQuotaResponseProto VOID_SETQUOTA_RESPONSE = + SetQuotaResponseProto.newBuilder().build(); - private static final FsyncResponseProto VOID_FSYNC_RESPONSE = - FsyncResponseProto.newBuilder().build(); + protected static final FsyncResponseProto VOID_FSYNC_RESPONSE = + FsyncResponseProto.newBuilder().build(); - private static final SetTimesResponseProto VOID_SETTIMES_RESPONSE = - SetTimesResponseProto.newBuilder().build(); + protected static final SetTimesResponseProto VOID_SETTIMES_RESPONSE = + SetTimesResponseProto.newBuilder().build(); - private static final CreateSymlinkResponseProto VOID_CREATESYMLINK_RESPONSE = - CreateSymlinkResponseProto.newBuilder().build(); + protected static final CreateSymlinkResponseProto VOID_CREATESYMLINK_RESPONSE = + CreateSymlinkResponseProto.newBuilder().build(); - private static final UpdatePipelineResponseProto - VOID_UPDATEPIPELINE_RESPONSE = - UpdatePipelineResponseProto.newBuilder().build(); + protected static final UpdatePipelineResponseProto VOID_UPDATEPIPELINE_RESPONSE = + UpdatePipelineResponseProto.newBuilder().build(); - private static final CancelDelegationTokenResponseProto - VOID_CANCELDELEGATIONTOKEN_RESPONSE = - CancelDelegationTokenResponseProto.newBuilder().build(); + protected static final CancelDelegationTokenResponseProto VOID_CANCELDELEGATIONTOKEN_RESPONSE = + CancelDelegationTokenResponseProto.newBuilder().build(); - private static final SetBalancerBandwidthResponseProto - VOID_SETBALANCERBANDWIDTH_RESPONSE = - SetBalancerBandwidthResponseProto.newBuilder().build(); + protected static final SetBalancerBandwidthResponseProto VOID_SETBALANCERBANDWIDTH_RESPONSE = + SetBalancerBandwidthResponseProto.newBuilder().build(); - private static final SetAclResponseProto - VOID_SETACL_RESPONSE = SetAclResponseProto.getDefaultInstance(); + protected static final SetAclResponseProto VOID_SETACL_RESPONSE = + SetAclResponseProto.getDefaultInstance(); - private static final ModifyAclEntriesResponseProto - VOID_MODIFYACLENTRIES_RESPONSE = ModifyAclEntriesResponseProto - .getDefaultInstance(); + protected static final ModifyAclEntriesResponseProto VOID_MODIFYACLENTRIES_RESPONSE = + ModifyAclEntriesResponseProto.getDefaultInstance(); - private static final RemoveAclEntriesResponseProto - VOID_REMOVEACLENTRIES_RESPONSE = RemoveAclEntriesResponseProto - .getDefaultInstance(); + protected static final RemoveAclEntriesResponseProto VOID_REMOVEACLENTRIES_RESPONSE = + RemoveAclEntriesResponseProto.getDefaultInstance(); - private static final RemoveDefaultAclResponseProto - VOID_REMOVEDEFAULTACL_RESPONSE = RemoveDefaultAclResponseProto - .getDefaultInstance(); + protected static final RemoveDefaultAclResponseProto VOID_REMOVEDEFAULTACL_RESPONSE = + RemoveDefaultAclResponseProto.getDefaultInstance(); - private static final RemoveAclResponseProto - VOID_REMOVEACL_RESPONSE = RemoveAclResponseProto.getDefaultInstance(); - - private static final SetXAttrResponseProto - VOID_SETXATTR_RESPONSE = SetXAttrResponseProto.getDefaultInstance(); - - private static final RemoveXAttrResponseProto - VOID_REMOVEXATTR_RESPONSE = RemoveXAttrResponseProto.getDefaultInstance(); + protected static final RemoveAclResponseProto VOID_REMOVEACL_RESPONSE = + RemoveAclResponseProto.getDefaultInstance(); - private static final CheckAccessResponseProto - VOID_CHECKACCESS_RESPONSE = CheckAccessResponseProto.getDefaultInstance(); + protected static final SetXAttrResponseProto VOID_SETXATTR_RESPONSE = + SetXAttrResponseProto.getDefaultInstance(); - private static final SatisfyStoragePolicyResponseProto + protected static final RemoveXAttrResponseProto VOID_REMOVEXATTR_RESPONSE = + RemoveXAttrResponseProto.getDefaultInstance(); + + protected static final CheckAccessResponseProto VOID_CHECKACCESS_RESPONSE = + CheckAccessResponseProto.getDefaultInstance(); + + protected static final SatisfyStoragePolicyResponseProto VOID_SATISFYSTORAGEPOLICY_RESPONSE = SatisfyStoragePolicyResponseProto .getDefaultInstance(); - /** - * Constructor - * + /** Constructor. * @param server - the NN server * @throws IOException */ @@ -730,7 +722,7 @@ public TruncateResponseProto truncate(RpcController controller, @Override public DeleteResponseProto delete(RpcController controller, - DeleteRequestProto req) throws ServiceException { + DeleteRequestProto req) throws ServiceException { try { boolean result = server.delete(req.getSrc(), req.getRecursive()); return DeleteResponseProto.newBuilder().setResult(result).build(); @@ -1474,7 +1466,7 @@ public ListCacheDirectivesResponseProto listCacheDirectives( CacheDirectiveInfo filter = PBHelperClient.convert(request.getFilter()); BatchedEntries entries = - server.listCacheDirectives(request.getPrevId(), filter); + server.listCacheDirectives(request.getPrevId(), filter); ListCacheDirectivesResponseProto.Builder builder = ListCacheDirectivesResponseProto.newBuilder(); builder.setHasMore(entries.hasMore()); @@ -1525,9 +1517,9 @@ public ListCachePoolsResponseProto listCachePools(RpcController controller, ListCachePoolsRequestProto request) throws ServiceException { try { BatchedEntries entries = - server.listCachePools(request.getPrevPoolName()); + server.listCachePools(request.getPrevPoolName()); ListCachePoolsResponseProto.Builder responseBuilder = - ListCachePoolsResponseProto.newBuilder(); + ListCachePoolsResponseProto.newBuilder(); responseBuilder.setHasMore(entries.hasMore()); for (int i=0, n=entries.size(); i entries = server .listEncryptionZones(req.getId()); @@ -1755,7 +1747,7 @@ public GetXAttrsResponseProto getXAttrs(RpcController controller, @Override public ListXAttrsResponseProto listXAttrs(RpcController controller, - ListXAttrsRequestProto req) throws ServiceException { + ListXAttrsRequestProto req) throws ServiceException { try { return PBHelperClient.convertListXAttrsResponse(server.listXAttrs(req.getSrc())); } catch (IOException e) { @@ -1776,7 +1768,7 @@ public RemoveXAttrResponseProto removeXAttr(RpcController controller, @Override public CheckAccessResponseProto checkAccess(RpcController controller, - CheckAccessRequestProto req) throws ServiceException { + CheckAccessRequestProto req) throws ServiceException { try { server.checkAccess(req.getPath(), PBHelperClient.convert(req.getMode())); } catch (IOException e) { @@ -1869,8 +1861,8 @@ public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies(RpcControl GetErasureCodingPoliciesRequestProto request) throws ServiceException { try { ErasureCodingPolicyInfo[] ecpInfos = server.getErasureCodingPolicies(); - GetErasureCodingPoliciesResponseProto.Builder resBuilder = GetErasureCodingPoliciesResponseProto - .newBuilder(); + GetErasureCodingPoliciesResponseProto.Builder resBuilder = + GetErasureCodingPoliciesResponseProto.newBuilder(); for (ErasureCodingPolicyInfo info : ecpInfos) { resBuilder.addEcPolicies( PBHelperClient.convertErasureCodingPolicy(info)); @@ -1965,7 +1957,8 @@ public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController GetErasureCodingPolicyRequestProto request) throws ServiceException { try { ErasureCodingPolicy ecPolicy = server.getErasureCodingPolicy(request.getSrc()); - GetErasureCodingPolicyResponseProto.Builder builder = GetErasureCodingPolicyResponseProto.newBuilder(); + GetErasureCodingPolicyResponseProto.Builder builder = + GetErasureCodingPolicyResponseProto.newBuilder(); if (ecPolicy != null) { builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy)); } 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;