From 65c5bc401bd017671e26c67ff49b025d098ae747 Mon Sep 17 00:00:00 2001 From: Jian Zhang <1361320460@qq.com> Date: Mon, 3 Jun 2024 21:06:30 +0800 Subject: [PATCH 1/4] async util & ut async util & ut async util & ut async util javadoc async util javadoc asyncTry feat test class javadoc test Finally & currentMethod --- .../router/async/ApplyFunction.java | 80 ++++ .../server/federation/router/async/Async.java | 85 +++++ .../router/async/AsyncApplyFunction.java | 138 +++++++ .../router/async/AsyncCatchFunction.java | 73 ++++ .../router/async/AsyncForEachRun.java | 97 +++++ .../federation/router/async/AsyncRun.java | 35 ++ .../federation/router/async/AsyncUtil.java | 357 ++++++++++++++++++ .../federation/router/async/BiFunction.java | 25 ++ .../router/async/CatchFunction.java | 49 +++ .../router/async/FinallyFunction.java | 42 +++ .../federation/router/async/package-info.java | 35 ++ .../federation/router/async/AsyncClass.java | 241 ++++++++++++ .../federation/router/async/BaseClass.java | 64 ++++ .../federation/router/async/SyncClass.java | 189 ++++++++++ .../router/async/TestAsyncUtil.java | 259 +++++++++++++ 15 files changed, 1769 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/FinallyFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/package-info.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/BaseClass.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/SyncClass.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestAsyncUtil.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.java new file mode 100644 index 0000000000000..130d3e8c49fe9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.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.server.federation.router.async; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * 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}. + * + * @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 new CompletionException(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 new CompletionException(e); + } + }, executor); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java new file mode 100644 index 0000000000000..137609ac52fc8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java @@ -0,0 +1,85 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** + * 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); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java new file mode 100644 index 0000000000000..be8ed24523ee9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java @@ -0,0 +1,138 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * 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.

+ * + * @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(); + } + + + 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 new CompletionException(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 new CompletionException(e); + } + }, executor); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java new file mode 100644 index 0000000000000..c2d9c666d40fb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java @@ -0,0 +1,73 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; + +@FunctionalInterface +public interface AsyncCatchFunction + extends CatchFunction { + + void applyAsync(R r, E e) throws IOException; + + @Override + default R apply(R r, E e) throws IOException { + applyAsync(r, e); + return result(); + } + + default CompletableFuture async(R r, E e) throws IOException { + applyAsync(r, e); + CompletableFuture completableFuture = getCurCompletableFuture(); + assert completableFuture != null; + return completableFuture; + } + + @Override + default CompletableFuture apply( + CompletableFuture in, Class eClazz) { + CompletableFuture result = new CompletableFuture<>(); + in.handle((r, e) -> { + if (e == null) { + result.complete(r); + return r; + } + Throwable cause = e.getCause(); + + if (eClazz.isInstance(cause)) { + try { + async(r, (E) cause).handle((r1, ex) -> { + if (ex != null) { + result.completeExceptionally(ex); + } else { + result.complete(r1); + } + return null; + }); + } catch (IOException ioe) { + result.completeExceptionally(ioe); + } + } else { + result.completeExceptionally(cause); + } + return r; + }); + return result; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java new file mode 100644 index 0000000000000..4fe97d15550c9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java @@ -0,0 +1,97 @@ +/** + * 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 java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +public class AsyncForEachRun implements AsyncRun { + + private boolean satisfy = false; + private Iterator iterator; + private I now; + private final CompletableFuture result = new CompletableFuture<>(); + private AsyncApplyFunction asyncFunction; + private BiFunction, T, R> thenApply; + + @Override + public void run() { + try { + doOnce(null); + } catch (IOException ioe) { + result.completeExceptionally(ioe); + } + setCurCompletableFuture(result); + } + + private void doOnce(R ret) throws IOException { + if (!iterator.hasNext()) { + result.complete(ret); + return; + } + now = iterator.next(); + CompletableFuture completableFuture = asyncFunction.async(now); + completableFuture.thenApply(t -> { + R r = null; + try { + r = thenApply.apply(AsyncForEachRun.this, t); + } catch (IOException e) { + result.completeExceptionally(e); + return null; + } + if (satisfy) { + result.complete(r); + return null; + } + try { + doOnce(r); + } catch (IOException e) { + throw new CompletionException(e); + } + return null; + }).exceptionally(e -> + result.completeExceptionally(e.getCause())); + } + + public I getNow() { + return now; + } + + public void breakNow() { + satisfy = true; + } + + public AsyncForEachRun forEach(Iterator forEach) { + this.iterator = forEach; + return this; + } + + public AsyncForEachRun asyncDo(AsyncApplyFunction asyncDo) { + this.asyncFunction = asyncDo; + return this; + } + + public AsyncForEachRun then( + BiFunction, T, R> then) { + this.thenApply = then; + return this; + } +} + diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.java new file mode 100644 index 0000000000000..8ab1a6c2feaeb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.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. + */ +package org.apache.hadoop.hdfs.server.federation.router.async; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +@FunctionalInterface +public interface AsyncRun extends Async { + + void run() throws IOException; + + 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/AsyncUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java new file mode 100644 index 0000000000000..4bed7eb58dd8f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java @@ -0,0 +1,357 @@ +/** + * 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 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.Async.CUR_COMPLETABLE_FUTURE; + +/** + * 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 + */ +public final class AsyncUtil { + private static final Boolean BOOLEAN_RESULT = false; + private static final Long LONG_RESULT = -1L; + 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}, + *
  • {@code null} for any other type. + *
+ */ + public static R asyncReturn(Class clazz) { + if (clazz == null) { + return null; + } + if (clazz.equals(Boolean.class)) { + return (R) BOOLEAN_RESULT; + } else if (clazz.equals(Long.class)) { + return (R) LONG_RESULT; + } + return (R) NULL_RESULT; + } + + 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 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(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); + } + + /** + * Applies an asynchronous operation to each element in an Iterator, + * using the provided functions to process each element and to + * determine the final result. + * + *

This method is useful for performing operations on a collection + * of items in a non-blocking way. It iterates over each element + * provided by the Iterator, applies the asyncDo operation to it, + * and then uses the then function to process the results.

+ * + * @param forEach The Iterator over the elements to process. + * @param asyncDo The asynchronous operation to apply to each element. + * It implements {@link AsyncApplyFunction}. + * @param then The function to apply after all elements have been + * processed by asyncDo. It implements + * {@link BiFunction} and determines how to combine + * the results into a final output. + * @param The type of the elements being iterated over. + * @param The type of the intermediate result from asyncDo. + * @param The type of the final result after applying the then function. + * @see AsyncApplyFunction + * @see BiFunction + */ + public static void asyncForEach( + Iterator forEach, AsyncApplyFunction asyncDo, + BiFunction, T, R> then) { + AsyncForEachRun asyncForEachRun = new AsyncForEachRun<>(); + asyncForEachRun.forEach(forEach).asyncDo(asyncDo).then(then).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(e); + } + completableFutures[i++] = future; + } + CompletableFuture

result = CompletableFuture.allOf(completableFutures) + .handle((unused, throwable) -> then.apply(completableFutures)); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } +} + diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java new file mode 100644 index 0000000000000..1a1686b66b4e2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java @@ -0,0 +1,25 @@ +/** + * 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 java.io.IOException; + +@FunctionalInterface +public interface BiFunction { + R apply(T t, P p) throws IOException; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java new file mode 100644 index 0000000000000..ae492ec5738e2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java @@ -0,0 +1,49 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +@FunctionalInterface +public interface CatchFunction + extends Async{ + R apply(R r, E e) throws IOException; + + default CompletableFuture apply( + CompletableFuture in, Class eClazz) { + return in.handle((r, e) -> { + if (e == null) { + return r; + } + Throwable cause = e.getCause(); + assert cause != null; + if (eClazz.isInstance(cause)) { + try { + return CatchFunction.this.apply(r, (E) cause); + } catch (IOException ioe) { + throw new CompletionException(ioe); + } + } else { + throw new CompletionException(cause); + } + }); + } + +} \ 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/FinallyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/FinallyFunction.java new file mode 100644 index 0000000000000..9c3f8365dfc60 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/FinallyFunction.java @@ -0,0 +1,42 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +public interface FinallyFunction { + R apply(R r) throws IOException; + + default CompletableFuture apply(CompletableFuture in) { + return in.handle((r, e) -> { + try { + R ret = apply(r); + if (e != null) { + throw new CompletionException(e.getCause()); + } else { + return ret; + } + } catch (IOException ex) { + throw new CompletionException(ex); + } + }); + } +} + 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..48fd0ad89abfe --- /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,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; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java new file mode 100644 index 0000000000000..eb1421e116561 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java @@ -0,0 +1,241 @@ +/** + * 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.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.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncCatch; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncCurrent; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncFinally; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncForEach; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncThrowException; +import static org.apache.hadoop.hdfs.server.federation.router.async.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(TestAsyncUtil.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 forEachMethod(List list) { + StringBuilder result = new StringBuilder(); + asyncForEach(list.iterator(), + input -> timeConsumingMethod(input), + (forEachRun, 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(), + input -> timeConsumingMethod(input), + (forEachRun, res) -> { + if (res.equals("[2]")) { + forEachRun.breakNow(); + } else { + result.append("forEach" + res + ","); + } + return result.toString(); + }); + return asyncReturn(String.class); + } + + @Override + public String forEachBreakByExceptionMethod(List list) { + StringBuilder result = new StringBuilder(); + boolean[] breakNow = {false}; + asyncForEach(list.iterator(), + input -> { + asyncTry(() -> { + applyMethod(input, true); + asyncApply(res -> { + result.append("forEach" + res + ","); + return res; + }); + }); + asyncCatch((res, e) -> { + if (e instanceof IOException) { + result.append(e + ","); + } else if (e instanceof RuntimeException) { + breakNow[0] = true; + } + return res; + }, Exception.class); + }, + (forEachRun, res) -> { + if (breakNow[0]) { + forEachRun.breakNow(); + } + return result.toString(); + }); + 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/BaseClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/BaseClass.java new file mode 100644 index 0000000000000..2181ea3ce6607 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/BaseClass.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.server.federation.router.async; + +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 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/SyncClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/SyncClass.java new file mode 100644 index 0000000000000..f72fd480fea0e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/SyncClass.java @@ -0,0 +1,189 @@ +/** + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 static final Logger LOG = + LoggerFactory.getLogger(TestAsyncUtil.class); + 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 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/TestAsyncUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestAsyncUtil.java new file mode 100644 index 0000000000000..4ca67ea600b45 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestAsyncUtil.java @@ -0,0 +1,259 @@ +/** + * 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.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 + * settings. + * + * @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 setting) { + if (setting.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 setting) + throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.applyMethod(1); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("applyMethod[1]", result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testApplyException(ExecutionMode setting) + throws Exception { + setUp(setting); + 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 testApplyThenApplyMethod(ExecutionMode setting) throws Exception { + setUp(setting); + 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", setting, cost); + + start = Time.monotonicNow(); + result = baseClass.applyThenApplyMethod(3); + cost = Time.monotonicNow() - start; + checkResult("[3]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", setting, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCatchThenApplyMethod(ExecutionMode setting) throws Exception { + setUp(setting); + 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", setting, cost); + + start = Time.monotonicNow(); + result = baseClass.applyCatchThenApplyMethod(0); + cost = Time.monotonicNow() - start; + checkResult("[0]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", setting, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCatchFinallyMethod(ExecutionMode setting) throws Exception { + setUp(setting); + 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", setting, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEach(ExecutionMode setting) throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.forEachMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("forEach[1],forEach[2],forEach[3],", result, + TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEachBreak(ExecutionMode setting) throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.forEachBreakMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("forEach[1],", result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEachBreakByException(ExecutionMode setting) + throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.forEachBreakByExceptionMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("forEach[1],java.io.IOException: input 2 exception,", + result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCurrentMethod(ExecutionMode setting) + throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.currentMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, 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; + }); + } + } +} \ No newline at end of file From 04c14cd0387e99912ffa0fa24c07d4741ff2796a Mon Sep 17 00:00:00 2001 From: Jian Zhang <1361320460@qq.com> Date: Mon, 3 Jun 2024 21:06:30 +0800 Subject: [PATCH 2/4] async util & ut async util & ut async util & ut async util javadoc async util javadoc asyncTry feat test class javadoc test Finally & currentMethod --- .../router/async/ApplyFunction.java | 80 ++++ .../server/federation/router/async/Async.java | 85 +++++ .../router/async/AsyncApplyFunction.java | 138 +++++++ .../router/async/AsyncCatchFunction.java | 73 ++++ .../router/async/AsyncForEachRun.java | 97 +++++ .../federation/router/async/AsyncRun.java | 35 ++ .../federation/router/async/AsyncUtil.java | 357 ++++++++++++++++++ .../federation/router/async/BiFunction.java | 25 ++ .../router/async/CatchFunction.java | 49 +++ .../router/async/FinallyFunction.java | 42 +++ .../federation/router/async/package-info.java | 35 ++ .../federation/router/async/AsyncClass.java | 241 ++++++++++++ .../federation/router/async/BaseClass.java | 64 ++++ .../federation/router/async/SyncClass.java | 189 ++++++++++ .../router/async/TestAsyncUtil.java | 259 +++++++++++++ 15 files changed, 1769 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/FinallyFunction.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/package-info.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/BaseClass.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/SyncClass.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestAsyncUtil.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.java new file mode 100644 index 0000000000000..130d3e8c49fe9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/ApplyFunction.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.server.federation.router.async; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * 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}. + * + * @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 new CompletionException(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 new CompletionException(e); + } + }, executor); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java new file mode 100644 index 0000000000000..137609ac52fc8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/Async.java @@ -0,0 +1,85 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** + * 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); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java new file mode 100644 index 0000000000000..be8ed24523ee9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncApplyFunction.java @@ -0,0 +1,138 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * 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.

+ * + * @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(); + } + + + 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 new CompletionException(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 new CompletionException(e); + } + }, executor); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java new file mode 100644 index 0000000000000..c2d9c666d40fb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncCatchFunction.java @@ -0,0 +1,73 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; + +@FunctionalInterface +public interface AsyncCatchFunction + extends CatchFunction { + + void applyAsync(R r, E e) throws IOException; + + @Override + default R apply(R r, E e) throws IOException { + applyAsync(r, e); + return result(); + } + + default CompletableFuture async(R r, E e) throws IOException { + applyAsync(r, e); + CompletableFuture completableFuture = getCurCompletableFuture(); + assert completableFuture != null; + return completableFuture; + } + + @Override + default CompletableFuture apply( + CompletableFuture in, Class eClazz) { + CompletableFuture result = new CompletableFuture<>(); + in.handle((r, e) -> { + if (e == null) { + result.complete(r); + return r; + } + Throwable cause = e.getCause(); + + if (eClazz.isInstance(cause)) { + try { + async(r, (E) cause).handle((r1, ex) -> { + if (ex != null) { + result.completeExceptionally(ex); + } else { + result.complete(r1); + } + return null; + }); + } catch (IOException ioe) { + result.completeExceptionally(ioe); + } + } else { + result.completeExceptionally(cause); + } + return r; + }); + return result; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java new file mode 100644 index 0000000000000..4fe97d15550c9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java @@ -0,0 +1,97 @@ +/** + * 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 java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +public class AsyncForEachRun implements AsyncRun { + + private boolean satisfy = false; + private Iterator iterator; + private I now; + private final CompletableFuture result = new CompletableFuture<>(); + private AsyncApplyFunction asyncFunction; + private BiFunction, T, R> thenApply; + + @Override + public void run() { + try { + doOnce(null); + } catch (IOException ioe) { + result.completeExceptionally(ioe); + } + setCurCompletableFuture(result); + } + + private void doOnce(R ret) throws IOException { + if (!iterator.hasNext()) { + result.complete(ret); + return; + } + now = iterator.next(); + CompletableFuture completableFuture = asyncFunction.async(now); + completableFuture.thenApply(t -> { + R r = null; + try { + r = thenApply.apply(AsyncForEachRun.this, t); + } catch (IOException e) { + result.completeExceptionally(e); + return null; + } + if (satisfy) { + result.complete(r); + return null; + } + try { + doOnce(r); + } catch (IOException e) { + throw new CompletionException(e); + } + return null; + }).exceptionally(e -> + result.completeExceptionally(e.getCause())); + } + + public I getNow() { + return now; + } + + public void breakNow() { + satisfy = true; + } + + public AsyncForEachRun forEach(Iterator forEach) { + this.iterator = forEach; + return this; + } + + public AsyncForEachRun asyncDo(AsyncApplyFunction asyncDo) { + this.asyncFunction = asyncDo; + return this; + } + + public AsyncForEachRun then( + BiFunction, T, R> then) { + this.thenApply = then; + return this; + } +} + diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.java new file mode 100644 index 0000000000000..8ab1a6c2feaeb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncRun.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. + */ +package org.apache.hadoop.hdfs.server.federation.router.async; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +@FunctionalInterface +public interface AsyncRun extends Async { + + void run() throws IOException; + + 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/AsyncUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java new file mode 100644 index 0000000000000..4bed7eb58dd8f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java @@ -0,0 +1,357 @@ +/** + * 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 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.Async.CUR_COMPLETABLE_FUTURE; + +/** + * 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 + */ +public final class AsyncUtil { + private static final Boolean BOOLEAN_RESULT = false; + private static final Long LONG_RESULT = -1L; + 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}, + *
  • {@code null} for any other type. + *
+ */ + public static R asyncReturn(Class clazz) { + if (clazz == null) { + return null; + } + if (clazz.equals(Boolean.class)) { + return (R) BOOLEAN_RESULT; + } else if (clazz.equals(Long.class)) { + return (R) LONG_RESULT; + } + return (R) NULL_RESULT; + } + + 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 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(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); + } + + /** + * Applies an asynchronous operation to each element in an Iterator, + * using the provided functions to process each element and to + * determine the final result. + * + *

This method is useful for performing operations on a collection + * of items in a non-blocking way. It iterates over each element + * provided by the Iterator, applies the asyncDo operation to it, + * and then uses the then function to process the results.

+ * + * @param forEach The Iterator over the elements to process. + * @param asyncDo The asynchronous operation to apply to each element. + * It implements {@link AsyncApplyFunction}. + * @param then The function to apply after all elements have been + * processed by asyncDo. It implements + * {@link BiFunction} and determines how to combine + * the results into a final output. + * @param The type of the elements being iterated over. + * @param The type of the intermediate result from asyncDo. + * @param The type of the final result after applying the then function. + * @see AsyncApplyFunction + * @see BiFunction + */ + public static void asyncForEach( + Iterator forEach, AsyncApplyFunction asyncDo, + BiFunction, T, R> then) { + AsyncForEachRun asyncForEachRun = new AsyncForEachRun<>(); + asyncForEachRun.forEach(forEach).asyncDo(asyncDo).then(then).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(e); + } + completableFutures[i++] = future; + } + CompletableFuture

result = CompletableFuture.allOf(completableFutures) + .handle((unused, throwable) -> then.apply(completableFutures)); + CUR_COMPLETABLE_FUTURE.set((CompletableFuture) result); + } +} + diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java new file mode 100644 index 0000000000000..1a1686b66b4e2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/BiFunction.java @@ -0,0 +1,25 @@ +/** + * 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 java.io.IOException; + +@FunctionalInterface +public interface BiFunction { + R apply(T t, P p) throws IOException; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java new file mode 100644 index 0000000000000..ae492ec5738e2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/CatchFunction.java @@ -0,0 +1,49 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +@FunctionalInterface +public interface CatchFunction + extends Async{ + R apply(R r, E e) throws IOException; + + default CompletableFuture apply( + CompletableFuture in, Class eClazz) { + return in.handle((r, e) -> { + if (e == null) { + return r; + } + Throwable cause = e.getCause(); + assert cause != null; + if (eClazz.isInstance(cause)) { + try { + return CatchFunction.this.apply(r, (E) cause); + } catch (IOException ioe) { + throw new CompletionException(ioe); + } + } else { + throw new CompletionException(cause); + } + }); + } + +} \ 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/FinallyFunction.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/FinallyFunction.java new file mode 100644 index 0000000000000..9c3f8365dfc60 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/FinallyFunction.java @@ -0,0 +1,42 @@ +/** + * 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 java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +public interface FinallyFunction { + R apply(R r) throws IOException; + + default CompletableFuture apply(CompletableFuture in) { + return in.handle((r, e) -> { + try { + R ret = apply(r); + if (e != null) { + throw new CompletionException(e.getCause()); + } else { + return ret; + } + } catch (IOException ex) { + throw new CompletionException(ex); + } + }); + } +} + 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..48fd0ad89abfe --- /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,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; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java new file mode 100644 index 0000000000000..eb1421e116561 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncClass.java @@ -0,0 +1,241 @@ +/** + * 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.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.AsyncUtil.asyncApply; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncCatch; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncComplete; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncCurrent; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncFinally; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncForEach; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncReturn; +import static org.apache.hadoop.hdfs.server.federation.router.async.AsyncUtil.asyncThrowException; +import static org.apache.hadoop.hdfs.server.federation.router.async.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(TestAsyncUtil.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 forEachMethod(List list) { + StringBuilder result = new StringBuilder(); + asyncForEach(list.iterator(), + input -> timeConsumingMethod(input), + (forEachRun, 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(), + input -> timeConsumingMethod(input), + (forEachRun, res) -> { + if (res.equals("[2]")) { + forEachRun.breakNow(); + } else { + result.append("forEach" + res + ","); + } + return result.toString(); + }); + return asyncReturn(String.class); + } + + @Override + public String forEachBreakByExceptionMethod(List list) { + StringBuilder result = new StringBuilder(); + boolean[] breakNow = {false}; + asyncForEach(list.iterator(), + input -> { + asyncTry(() -> { + applyMethod(input, true); + asyncApply(res -> { + result.append("forEach" + res + ","); + return res; + }); + }); + asyncCatch((res, e) -> { + if (e instanceof IOException) { + result.append(e + ","); + } else if (e instanceof RuntimeException) { + breakNow[0] = true; + } + return res; + }, Exception.class); + }, + (forEachRun, res) -> { + if (breakNow[0]) { + forEachRun.breakNow(); + } + return result.toString(); + }); + 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/BaseClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/BaseClass.java new file mode 100644 index 0000000000000..2181ea3ce6607 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/BaseClass.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.server.federation.router.async; + +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 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/SyncClass.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/SyncClass.java new file mode 100644 index 0000000000000..f72fd480fea0e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/SyncClass.java @@ -0,0 +1,189 @@ +/** + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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 static final Logger LOG = + LoggerFactory.getLogger(TestAsyncUtil.class); + 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 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/TestAsyncUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestAsyncUtil.java new file mode 100644 index 0000000000000..4ca67ea600b45 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestAsyncUtil.java @@ -0,0 +1,259 @@ +/** + * 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.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 + * settings. + * + * @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 setting) { + if (setting.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 setting) + throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.applyMethod(1); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("applyMethod[1]", result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testApplyException(ExecutionMode setting) + throws Exception { + setUp(setting); + 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 testApplyThenApplyMethod(ExecutionMode setting) throws Exception { + setUp(setting); + 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", setting, cost); + + start = Time.monotonicNow(); + result = baseClass.applyThenApplyMethod(3); + cost = Time.monotonicNow() - start; + checkResult("[3]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", setting, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCatchThenApplyMethod(ExecutionMode setting) throws Exception { + setUp(setting); + 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", setting, cost); + + start = Time.monotonicNow(); + result = baseClass.applyCatchThenApplyMethod(0); + cost = Time.monotonicNow() - start; + checkResult("[0]", result, TIME_CONSUMING, cost); + LOG.info("[{}] main thread cost: {} ms", setting, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCatchFinallyMethod(ExecutionMode setting) throws Exception { + setUp(setting); + 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", setting, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEach(ExecutionMode setting) throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.forEachMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("forEach[1],forEach[2],forEach[3],", result, + TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEachBreak(ExecutionMode setting) throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.forEachBreakMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("forEach[1],", result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testForEachBreakByException(ExecutionMode setting) + throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.forEachBreakByExceptionMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, cost); + checkResult("forEach[1],java.io.IOException: input 2 exception,", + result, TIME_CONSUMING, cost); + } + + @EnumSource(ExecutionMode.class) + @ParameterizedTest + public void testCurrentMethod(ExecutionMode setting) + throws Exception { + setUp(setting); + long start = Time.monotonicNow(); + String result = baseClass.currentMethod(Arrays.asList(1, 2, 3)); + long cost = Time.monotonicNow() - start; + LOG.info("[{}] main thread cost: {} ms", setting, 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; + }); + } + } +} \ No newline at end of file From 1b633451c123928cbd9681769584910092c8b3ca Mon Sep 17 00:00:00 2001 From: Jian Zhang <1361320460@qq.com> Date: Tue, 4 Jun 2024 21:55:26 +0800 Subject: [PATCH 3/4] router async rpc client router async rpc client --- .../java/org/apache/hadoop/ipc/Client.java | 7 + ...appingsProtocolClientSideTranslatorPB.java | 6 +- ...appingsProtocolClientSideTranslatorPB.java | 2 +- .../ClientNamenodeProtocolTranslatorPB.java | 26 +- .../protocolPB/AsyncRpcProtocolPBUtil.java | 73 + .../RouterClientProtocolTranslatorPB.java | 2707 +++++++++++++++++ ...erGetUserMappingsProtocolTranslatorPB.java | 57 + .../RouterNamenodeProtocolTranslatorPB.java | 283 ++ ...freshUserMappingsProtocolTranslatorPB.java | 72 + .../federation/router/ConnectionPool.java | 16 +- .../federation/router/RouterRpcClient.java | 20 +- .../NamenodeProtocolTranslatorPB.java | 10 +- 12 files changed, 3248 insertions(+), 31 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientProtocolTranslatorPB.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolTranslatorPB.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolTranslatorPB.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolTranslatorPB.java 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..5b715be661e4a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -96,6 +96,8 @@ public class Client implements AutoCloseable { private static final ThreadLocal retryCount = new ThreadLocal(); private static final ThreadLocal EXTERNAL_CALL_HANDLER = new ThreadLocal<>(); + public static final ThreadLocal> CALL_FUTURE_THREAD_LOCAL + = new ThreadLocal<>(); private static final ThreadLocal> ASYNC_RPC_RESPONSE = new ThreadLocal<>(); private static final ThreadLocal asynchronousMode = @@ -283,6 +285,7 @@ static class Call { boolean done; // true when call is done private final Object externalHandler; private AlignmentContext alignmentContext; + private CompletableFuture completableFuture; private Call(RPC.RpcKind rpcKind, Writable param) { this.rpcKind = rpcKind; @@ -304,6 +307,7 @@ private Call(RPC.RpcKind rpcKind, Writable param) { } this.externalHandler = EXTERNAL_CALL_HANDLER.get(); + this.completableFuture = CALL_FUTURE_THREAD_LOCAL.get(); } @Override @@ -322,6 +326,9 @@ protected synchronized void callComplete() { externalHandler.notify(); } } + if (completableFuture != null) { + completableFuture.complete(this); + } } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java index cb80d067bb8bc..fe9b3056c7544 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/protocolPB/RefreshUserMappingsProtocolClientSideTranslatorPB.java @@ -35,14 +35,14 @@ public class RefreshUserMappingsProtocolClientSideTranslatorPB implements ProtocolMetaInterface, RefreshUserMappingsProtocol, Closeable { /** RpcController is not used and hence is set to null */ - private final static RpcController NULL_CONTROLLER = null; + protected final static RpcController NULL_CONTROLLER = null; private final RefreshUserMappingsProtocolPB rpcProxy; - private final static RefreshUserToGroupsMappingsRequestProto + protected final static RefreshUserToGroupsMappingsRequestProto VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST = RefreshUserToGroupsMappingsRequestProto.newBuilder().build(); - private final static RefreshSuperUserGroupsConfigurationRequestProto + protected final static RefreshSuperUserGroupsConfigurationRequestProto VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST = RefreshSuperUserGroupsConfigurationRequestProto.newBuilder().build(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java index 8d8885adcba9a..43322b4563006 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/protocolPB/GetUserMappingsProtocolClientSideTranslatorPB.java @@ -36,7 +36,7 @@ public class GetUserMappingsProtocolClientSideTranslatorPB implements ProtocolMetaInterface, GetUserMappingsProtocol, Closeable { /** RpcController is not used and hence is set to null */ - private final static RpcController NULL_CONTROLLER = null; + protected final static RpcController NULL_CONTROLLER = null; private final GetUserMappingsProtocolPB rpcProxy; public GetUserMappingsProtocolClientSideTranslatorPB( diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java index 543f0a58e6ec6..28dc92341aab1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java @@ -272,49 +272,49 @@ @InterfaceStability.Stable public class ClientNamenodeProtocolTranslatorPB implements ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator { - final private ClientNamenodeProtocolPB rpcProxy; + final protected ClientNamenodeProtocolPB rpcProxy; - static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = + protected static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = GetServerDefaultsRequestProto.newBuilder().build(); - private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST = + protected final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST = GetFsStatusRequestProto.newBuilder().build(); - private final static GetFsReplicatedBlockStatsRequestProto + protected final static GetFsReplicatedBlockStatsRequestProto VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST = GetFsReplicatedBlockStatsRequestProto.newBuilder().build(); - private final static GetFsECBlockGroupStatsRequestProto + protected final static GetFsECBlockGroupStatsRequestProto VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST = GetFsECBlockGroupStatsRequestProto.newBuilder().build(); - private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = + protected final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = RollEditsRequestProto.getDefaultInstance(); - private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST = + protected final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST = RefreshNodesRequestProto.newBuilder().build(); - private final static FinalizeUpgradeRequestProto + protected final static FinalizeUpgradeRequestProto VOID_FINALIZE_UPGRADE_REQUEST = FinalizeUpgradeRequestProto.newBuilder().build(); - private final static UpgradeStatusRequestProto + protected final static UpgradeStatusRequestProto VOID_UPGRADE_STATUS_REQUEST = UpgradeStatusRequestProto.newBuilder().build(); - private final static GetDataEncryptionKeyRequestProto + protected final static GetDataEncryptionKeyRequestProto VOID_GET_DATA_ENCRYPTIONKEY_REQUEST = GetDataEncryptionKeyRequestProto.newBuilder().build(); - private final static GetStoragePoliciesRequestProto + protected final static GetStoragePoliciesRequestProto VOID_GET_STORAGE_POLICIES_REQUEST = GetStoragePoliciesRequestProto.newBuilder().build(); - private final static GetErasureCodingPoliciesRequestProto + protected final static GetErasureCodingPoliciesRequestProto VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto .newBuilder().build(); - private final static GetErasureCodingCodecsRequestProto + protected final static GetErasureCodingCodecsRequestProto VOID_GET_EC_CODEC_REQUEST = GetErasureCodingCodecsRequestProto .newBuilder().build(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java new file mode 100644 index 0000000000000..38c01e048c831 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/AsyncRpcProtocolPBUtil.java @@ -0,0 +1,73 @@ +/** + * 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.async.Async; +import org.apache.hadoop.ipc.CallerContext; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.ProtobufRpcEngine2; +import org.apache.hadoop.ipc.Server; +import org.apache.hadoop.ipc.internal.ShadedProtobufHelper; +import org.apache.hadoop.util.concurrent.AsyncGet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + + +import static org.apache.hadoop.ipc.internal.ShadedProtobufHelper.ipc; + +public final class AsyncRpcProtocolPBUtil { + public static final Logger LOG = LoggerFactory.getLogger(AsyncRpcProtocolPBUtil.class); + + private AsyncRpcProtocolPBUtil() {} + + public static AsyncGet asyncIpc( + ShadedProtobufHelper.IpcCall call) throws IOException { + CompletableFuture completableFuture = new CompletableFuture<>(); + Client.CALL_FUTURE_THREAD_LOCAL.set(completableFuture); + ipc(call); + return (AsyncGet) ProtobufRpcEngine2.getAsyncReturnMessage(); + } + + public static void asyncResponse(Response response) { + CompletableFuture callCompletableFuture = + (CompletableFuture) Client.CALL_FUTURE_THREAD_LOCAL.get(); + // transfer originCall & callerContext to worker threads of executor. + final Server.Call originCall = Server.getCurCall().get(); + final CallerContext originContext = CallerContext.getCurrent(); + CompletableFuture result = callCompletableFuture.thenApplyAsync(t -> { + try { + Server.getCurCall().set(originCall); + CallerContext.setCurrent(originContext); + return response.response(); + }catch (Exception e) { + throw new CompletionException(e); + } + }); + Async.CUR_COMPLETABLE_FUTURE.set(result); + } + + @FunctionalInterface + interface Response { + T response() throws Exception; + } +} \ No newline at end of file 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..b50f4c11ce2f8 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterClientProtocolTranslatorPB.java @@ -0,0 +1,2707 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.protocolPB; + +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; + +import java.util.Map; +import java.util.stream.Collectors; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.crypto.CryptoProtocolVersion; +import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries; +import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries; +import org.apache.hadoop.fs.CacheFlag; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FsServerDefaults; +import org.apache.hadoop.fs.Options.Rename; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.QuotaUsage; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.fs.XAttr; +import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.AclStatus; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.ha.HAServiceProtocol; +import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto; +import org.apache.hadoop.hdfs.AddBlockFlag; +import org.apache.hadoop.hdfs.inotify.EventBatchList; +import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; +import org.apache.hadoop.hdfs.protocol.BatchedDirectoryListing; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; +import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; +import org.apache.hadoop.hdfs.protocol.CachePoolEntry; +import org.apache.hadoop.hdfs.protocol.CachePoolInfo; +import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks; +import org.apache.hadoop.hdfs.protocol.DatanodeID; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.HdfsPartialListing; +import org.apache.hadoop.hdfs.protocol.ECBlockGroupStats; +import org.apache.hadoop.hdfs.protocol.ECTopologyVerifierResult; +import org.apache.hadoop.hdfs.protocol.EncryptionZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus; +import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType; +import org.apache.hadoop.hdfs.protocol.ReplicatedBlockStats; +import org.apache.hadoop.hdfs.protocol.OpenFileEntry; +import org.apache.hadoop.hdfs.protocol.OpenFilesIterator; +import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus; +import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; +import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing; +import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; +import org.apache.hadoop.hdfs.protocol.SnapshotStatus; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.GetAclStatusResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.ModifyAclEntriesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclEntriesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.RemoveDefaultAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.AclProtos.SetAclRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCacheDirectiveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolEntryProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CheckAccessResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CompleteResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ConcatResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateSymlinkResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DeleteSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.DisallowSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FinalizeUpgradeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.FsyncResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetAdditionalDatanodeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBatchedListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBatchedListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetBlockLocationsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetContentSummaryResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetCurrentEditLogTxidResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEnclosingRootResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsECBlockGroupStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsReplicatedBlockStatsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSlowDatanodeReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSlowDatanodeReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotDiffReportListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshotListingResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListOpenFilesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListOpenFilesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.OpenFilesBatchResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCacheDirectiveResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenewLeaseResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ReportBadBlocksResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetBalancerBandwidthResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetOwnerResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetPermissionResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetQuotaResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetReplicationResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSafeModeResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.TruncateResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UnsetStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SatisfyStoragePolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZoneProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListReencryptionStatusRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListReencryptionStatusResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ZoneReencryptionStatusProto; +import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptEncryptionZoneRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.AddErasureCodingPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.RemoveErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.EnableErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.DisableErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingCodecsResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.UnsetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.CodecProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BatchedDirectoryListingProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECTopologyResultForPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetECTopologyResultForPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto; +import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; +import org.apache.hadoop.io.EnumSetWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.ipc.RPC; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.ipc.RpcClientUtil; +import org.apache.hadoop.security.proto.SecurityProtos; +import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; +import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto; +import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto; +import org.apache.hadoop.security.token.Token; + +import org.apache.hadoop.thirdparty.protobuf.ByteString; +import org.apache.hadoop.thirdparty.protobuf.ServiceException; + +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.util.concurrent.AsyncGet; +import org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.Response; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; +import static org.apache.hadoop.ipc.internal.ShadedProtobufHelper.getRemoteException; +import static org.apache.hadoop.ipc.internal.ShadedProtobufHelper.ipc; + + +/** + * This class forwards NN's ClientProtocol calls as RPC calls to the NN server + * while translating from the parameter types used in ClientProtocol to the + * new PB types. + */ +@InterfaceAudience.Private +@InterfaceStability.Stable +public class RouterClientProtocolTranslatorPB extends ClientNamenodeProtocolTranslatorPB { + + public RouterClientProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) { + super(proxy); + } + + @Override + public void close() { + super.close(); + } + + @Override + public LocatedBlocks getBlockLocations(String src, long offset, long length) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBlockLocations(src, offset, length); + } + GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto + .newBuilder() + .setSrc(src) + .setOffset(offset) + .setLength(length) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getBlockLocations(null, req)); + asyncResponse(() -> { + GetBlockLocationsResponseProto resp = asyncGet.get(-1, null); + return resp.hasLocations() ? + PBHelperClient.convert(resp.getLocations()) : null; + }); + return null; + } + + @Override + public FsServerDefaults getServerDefaults() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getServerDefaults(); + } + GetServerDefaultsRequestProto req = VOID_GET_SERVER_DEFAULT_REQUEST; + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getServerDefaults(null, req)); + asyncResponse(() -> + PBHelperClient.convert(asyncGet.get(-1, null).getServerDefaults())); + return null; + } + + @Override + public HdfsFileStatus create( + String src, FsPermission masked, + String clientName, EnumSetWritable flag, + boolean createParent, short replication, long blockSize, + CryptoProtocolVersion[] supportedVersions, String ecPolicyName, + String storagePolicy) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.create( + src, masked, clientName, flag, createParent, replication, + blockSize, supportedVersions, ecPolicyName, storagePolicy); + } + + CreateRequestProto.Builder builder = CreateRequestProto.newBuilder() + .setSrc(src) + .setMasked(PBHelperClient.convert(masked)) + .setClientName(clientName) + .setCreateFlag(PBHelperClient.convertCreateFlag(flag)) + .setCreateParent(createParent) + .setReplication(replication) + .setBlockSize(blockSize); + if (ecPolicyName != null) { + builder.setEcPolicyName(ecPolicyName); + } + if (storagePolicy != null) { + builder.setStoragePolicy(storagePolicy); + } + FsPermission unmasked = masked.getUnmasked(); + if (unmasked != null) { + builder.setUnmasked(PBHelperClient.convert(unmasked)); + } + builder.addAllCryptoProtocolVersion( + PBHelperClient.convert(supportedVersions)); + CreateRequestProto req = builder.build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.create(null, req)); + asyncResponse(() -> { + CreateResponseProto res = asyncGet.get(-1, null); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + }); + return null; + } + + @Override + public boolean truncate(String src, long newLength, String clientName) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.truncate(src, newLength, clientName); + } + + TruncateRequestProto req = TruncateRequestProto.newBuilder() + .setSrc(src) + .setNewLength(newLength) + .setClientName(clientName) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.truncate(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + + @Override + public LastBlockWithStatus append(String src, String clientName, + EnumSetWritable flag) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.append(src, clientName, flag); + } + + AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src) + .setClientName(clientName).setFlag( + PBHelperClient.convertCreateFlag(flag)) + .build(); + + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.append(null, req)); + asyncResponse(() -> { + AppendResponseProto res = asyncGet.get(-1, null); + LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient + .convertLocatedBlockProto(res.getBlock()) : null; + HdfsFileStatus stat = (res.hasStat()) ? + PBHelperClient.convert(res.getStat()) : null; + return new LastBlockWithStatus(lastBlock, stat); + }); + return null; + } + + @Override + public boolean setReplication(String src, short replication) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.setReplication(src, replication); + } + + SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder() + .setSrc(src) + .setReplication(replication) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setReplication(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + + @Override + public void setPermission(String src, FsPermission permission) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setPermission(src, permission); + return; + } + + SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder() + .setSrc(src) + .setPermission(PBHelperClient.convert(permission)) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setPermission(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public void setOwner(String src, String username, String groupname) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.setOwner(src, username, groupname); + } + + SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder() + .setSrc(src); + if (username != null) { + req.setUsername(username); + } + if (groupname != null) { + req.setGroupname(groupname); + } + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setOwner(null, req.build())); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public void abandonBlock(ExtendedBlock b, long fileId, String src, + String holder) throws IOException { + if (!Client.isAsynchronousMode()) { + super.abandonBlock(b, fileId, src, holder); + } + AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder() + .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder) + .setFileId(fileId).build(); + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.abandonBlock(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public LocatedBlock addBlock( + String src, String clientName, + ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, + String[] favoredNodes, EnumSet addBlockFlags) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.addBlock(src, clientName, previous, excludeNodes, + fileId, favoredNodes, addBlockFlags); + } + AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder() + .setSrc(src).setClientName(clientName).setFileId(fileId); + if (previous != null) { + req.setPrevious(PBHelperClient.convert(previous)); + } + if (excludeNodes != null) { + req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes)); + } + if (favoredNodes != null) { + req.addAllFavoredNodes(Arrays.asList(favoredNodes)); + } + if (addBlockFlags != null) { + req.addAllFlags(PBHelperClient.convertAddBlockFlags( + addBlockFlags)); + } + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.addBlock(null, req.build())); + asyncResponse(() -> PBHelperClient.convertLocatedBlockProto( + asyncGet.get(-1, null).getBlock())); + return null; + } + + @Override + public LocatedBlock getAdditionalDatanode( + String src, long fileId, + ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs, + DatanodeInfo[] excludes, int numAdditionalNodes, String clientName) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getAdditionalDatanode(src, fileId, blk, existings, + existingStorageIDs, excludes, numAdditionalNodes, clientName); + } + GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto + .newBuilder() + .setSrc(src) + .setFileId(fileId) + .setBlk(PBHelperClient.convert(blk)) + .addAllExistings(PBHelperClient.convert(existings)) + .addAllExistingStorageUuids(Arrays.asList(existingStorageIDs)) + .addAllExcludes(PBHelperClient.convert(excludes)) + .setNumAdditionalNodes(numAdditionalNodes) + .setClientName(clientName) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getAdditionalDatanode(null, req)); + asyncResponse(() -> PBHelperClient.convertLocatedBlockProto( + asyncGet.get(-1, null).getBlock())); + return null; + } + + @Override + public boolean complete(String src, String clientName, + ExtendedBlock last, long fileId) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.complete(src, clientName, last, fileId); + } + CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder() + .setSrc(src) + .setClientName(clientName) + .setFileId(fileId); + if (last != null) { + req.setLast(PBHelperClient.convert(last)); + } + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.complete(null, req.build())); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + + @Override + public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { + if (!Client.isAsynchronousMode()) { + super.reportBadBlocks(blocks); + return; + } + ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder() + .addAllBlocks(Arrays.asList( + PBHelperClient.convertLocatedBlocks(blocks))) + .build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.reportBadBlocks(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public boolean rename(String src, String dst) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.rename(src, dst); + } + RenameRequestProto req = RenameRequestProto.newBuilder() + .setSrc(src) + .setDst(dst).build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rename(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getResult()); + return true; + } + + + @Override + public void rename2(String src, String dst, Rename... options) + throws IOException { + if (!Client.isAsynchronousMode()) { + super.rename2(src, dst, options); + return; + } + boolean overwrite = false; + boolean toTrash = false; + if (options != null) { + for (Rename option : options) { + if (option == Rename.OVERWRITE) { + overwrite = true; + } + if (option == Rename.TO_TRASH) { + toTrash = true; + } + } + } + Rename2RequestProto req = Rename2RequestProto.newBuilder(). + setSrc(src). + setDst(dst). + setOverwriteDest(overwrite). + setMoveToTrash(toTrash). + build(); + + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rename2(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } + + @Override + public void concat(String trg, String[] srcs) throws IOException { + ConcatRequestProto req = ConcatRequestProto.newBuilder(). + setTrg(trg). + addAllSrcs(Arrays.asList(srcs)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.concat(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.concat(null, req)); + } + } + + + @Override + public boolean delete(String src, boolean recursive) throws IOException { + DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src) + .setRecursive(recursive).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.delete(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.delete(null, req).getResult()); + } + + @Override + public boolean mkdirs(String src, FsPermission masked, boolean createParent) + throws IOException { + MkdirsRequestProto.Builder builder = MkdirsRequestProto.newBuilder() + .setSrc(src) + .setMasked(PBHelperClient.convert(masked)) + .setCreateParent(createParent); + FsPermission unmasked = masked.getUnmasked(); + if (unmasked != null) { + builder.setUnmasked(PBHelperClient.convert(unmasked)); + } + MkdirsRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.mkdirs(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.mkdirs(null, req)).getResult(); + } + + @Override + public DirectoryListing getListing(String src, byte[] startAfter, + boolean needLocation) throws IOException { + GetListingRequestProto req = GetListingRequestProto.newBuilder() + .setSrc(src) + .setStartAfter(ByteString.copyFrom(startAfter)) + .setNeedLocation(needLocation).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getListing(null, req)); + asyncResponse(() -> { + GetListingResponseProto result = asyncGet.get(-1, null); + if (result.hasDirList()) { + return PBHelperClient.convert(result.getDirList()); + } + return null; + }); + return null; + } + GetListingResponseProto result = ipc(() -> rpcProxy.getListing(null, req)); + if (result.hasDirList()) { + return PBHelperClient.convert(result.getDirList()); + } + return null; + } + + @Override + public BatchedDirectoryListing getBatchedListing( + String[] srcs, byte[] startAfter, boolean needLocation) + throws IOException { + GetBatchedListingRequestProto req = GetBatchedListingRequestProto + .newBuilder() + .addAllPaths(Arrays.asList(srcs)) + .setStartAfter(ByteString.copyFrom(startAfter)) + .setNeedLocation(needLocation).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getBatchedListing(null, req)); + asyncResponse(() -> { + GetBatchedListingResponseProto result = asyncGet.get(-1, null); + + if (result.getListingsCount() > 0) { + HdfsPartialListing[] listingArray = + new HdfsPartialListing[result.getListingsCount()]; + int listingIdx = 0; + for (BatchedDirectoryListingProto proto : result.getListingsList()) { + HdfsPartialListing listing; + if (proto.hasException()) { + HdfsProtos.RemoteExceptionProto reProto = proto.getException(); + RemoteException ex = new RemoteException( + reProto.getClassName(), reProto.getMessage()); + listing = new HdfsPartialListing(proto.getParentIdx(), ex); + } else { + List statuses = + PBHelperClient.convertHdfsFileStatus( + proto.getPartialListingList()); + listing = new HdfsPartialListing(proto.getParentIdx(), statuses); + } + listingArray[listingIdx++] = listing; + } + BatchedDirectoryListing batchedListing = + new BatchedDirectoryListing(listingArray, result.getHasMore(), + result.getStartAfter().toByteArray()); + return batchedListing; + } + return null; + }); + return null; + } + GetBatchedListingResponseProto result = + ipc(() -> rpcProxy.getBatchedListing(null, req)); + + if (result.getListingsCount() > 0) { + HdfsPartialListing[] listingArray = + new HdfsPartialListing[result.getListingsCount()]; + int listingIdx = 0; + for (BatchedDirectoryListingProto proto : result.getListingsList()) { + HdfsPartialListing listing; + if (proto.hasException()) { + HdfsProtos.RemoteExceptionProto reProto = proto.getException(); + RemoteException ex = new RemoteException( + reProto.getClassName(), reProto.getMessage()); + listing = new HdfsPartialListing(proto.getParentIdx(), ex); + } else { + List statuses = + PBHelperClient.convertHdfsFileStatus( + proto.getPartialListingList()); + listing = new HdfsPartialListing(proto.getParentIdx(), statuses); + } + listingArray[listingIdx++] = listing; + } + BatchedDirectoryListing batchedListing = + new BatchedDirectoryListing(listingArray, result.getHasMore(), + result.getStartAfter().toByteArray()); + return batchedListing; + } + return null; + } + + + @Override + public void renewLease(String clientName, List namespaces) + throws IOException { + RenewLeaseRequestProto.Builder builder = RenewLeaseRequestProto + .newBuilder().setClientName(clientName); + if (namespaces != null && !namespaces.isEmpty()) { + builder.addAllNamespaces(namespaces); + } + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.renewLease(null, builder.build())); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.renewLease(null, builder.build())); + } + } + + @Override + public boolean recoverLease(String src, String clientName) + throws IOException { + RecoverLeaseRequestProto req = RecoverLeaseRequestProto.newBuilder() + .setSrc(src) + .setClientName(clientName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.recoverLease(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.recoverLease(null, req)).getResult(); + } + + @Override + public long[] getStats() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFsStats(null, VOID_GET_FSSTATUS_REQUEST)); + asyncResponse(() -> PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getFsStats(null, + VOID_GET_FSSTATUS_REQUEST))); + } + + @Override + public ReplicatedBlockStats getReplicatedBlockStats() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getFsReplicatedBlockStats(null, + VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getFsReplicatedBlockStats(null, + VOID_GET_FS_REPLICATED_BLOCK_STATS_REQUEST))); + } + + @Override + public ECBlockGroupStats getECBlockGroupStats() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFsECBlockGroupStats(null, + VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getFsECBlockGroupStats(null, + VOID_GET_FS_ECBLOCKGROUP_STATS_REQUEST))); + } + + @Override + public DatanodeInfo[] getDatanodeReport(DatanodeReportType type) + throws IOException { + GetDatanodeReportRequestProto req = GetDatanodeReportRequestProto + .newBuilder() + .setType(PBHelperClient.convert(type)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDatanodeReport(null, req)); + asyncResponse(() -> + PBHelperClient.convert(asyncGet.get(-1, null).getDiList())); + return null; + } + return PBHelperClient.convert( + ipc(() -> rpcProxy.getDatanodeReport(null, req)).getDiList()); + } + + @Override + public DatanodeStorageReport[] getDatanodeStorageReport( + DatanodeReportType type) throws IOException { + final GetDatanodeStorageReportRequestProto req + = GetDatanodeStorageReportRequestProto.newBuilder() + .setType(PBHelperClient.convert(type)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDatanodeStorageReport(null, req)); + asyncResponse(() -> + PBHelperClient.convertDatanodeStorageReports( + asyncGet.get(-1, null).getDatanodeStorageReportsList())); + return null; + } + return PBHelperClient.convertDatanodeStorageReports( + ipc(() -> rpcProxy.getDatanodeStorageReport(null, req) + .getDatanodeStorageReportsList())); + } + + @Override + public long getPreferredBlockSize(String filename) throws IOException { + GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto + .newBuilder() + .setFilename(filename) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getPreferredBlockSize(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getBsize()); + return -1; + } + return ipc(() -> rpcProxy.getPreferredBlockSize(null, req)).getBsize(); + } + + @Override + public boolean setSafeMode(SafeModeAction action, boolean isChecked) + throws IOException { + SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder() + .setAction(PBHelperClient.convert(action)) + .setChecked(isChecked).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setSafeMode(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.setSafeMode(null, req)).getResult(); + } + + @Override + public boolean saveNamespace(long timeWindow, long txGap) throws IOException { + SaveNamespaceRequestProto req = SaveNamespaceRequestProto.newBuilder() + .setTimeWindow(timeWindow).setTxGap(txGap).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.saveNamespace(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getSaved()); + return true; + } + return ipc(() -> rpcProxy.saveNamespace(null, req)).getSaved(); + } + + @Override + public long rollEdits() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rollEdits(null, VOID_ROLLEDITS_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null).getNewSegmentTxId()); + return -1; + } + RollEditsResponseProto resp = ipc(() -> rpcProxy.rollEdits(null, + VOID_ROLLEDITS_REQUEST)); + return resp.getNewSegmentTxId(); + } + + @Override + public boolean restoreFailedStorage(String arg) throws IOException{ + RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto + .newBuilder() + .setArg(arg).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.restoreFailedStorage(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.restoreFailedStorage(null, req)).getResult(); + } + + @Override + public void refreshNodes() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.refreshNodes(null, VOID_REFRESH_NODES_REQUEST)); + } + } + + @Override + public void finalizeUpgrade() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.finalizeUpgrade(null, VOID_FINALIZE_UPGRADE_REQUEST)); + } + } + + @Override + public boolean upgradeStatus() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.upgradeStatus(null, VOID_UPGRADE_STATUS_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null).getUpgradeFinalized()); + return true; + } + final UpgradeStatusResponseProto proto = ipc(() -> rpcProxy.upgradeStatus( + null, VOID_UPGRADE_STATUS_REQUEST)); + return proto.getUpgradeFinalized(); + } + + @Override + public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) + throws IOException { + final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder() + .setAction(PBHelperClient.convert(action)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.rollingUpgrade(null, r)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null).getRollingUpgradeInfo())); + return null; + } + final RollingUpgradeResponseProto proto = + ipc(() -> rpcProxy.rollingUpgrade(null, r)); + if (proto.hasRollingUpgradeInfo()) { + return PBHelperClient.convert(proto.getRollingUpgradeInfo()); + } + return null; + } + + @Override + public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie) + throws IOException { + ListCorruptFileBlocksRequestProto.Builder req = + ListCorruptFileBlocksRequestProto.newBuilder().setPath(path); + if (cookie != null) { + req.setCookie(cookie); + } + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listCorruptFileBlocks(null, req.build())); + asyncResponse(() -> asyncGet.get(-1, null).getCorrupt()); + return null; + } + return PBHelperClient.convert( + ipc(() -> rpcProxy.listCorruptFileBlocks(null, req.build())).getCorrupt()); + } + + @Override + public void metaSave(String filename) throws IOException { + MetaSaveRequestProto req = MetaSaveRequestProto.newBuilder() + .setFilename(filename).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.metaSave(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.metaSave(null, req)); + } + } + + @Override + public HdfsFileStatus getFileInfo(String src) throws IOException { + GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder() + .setSrc(src) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFileInfo(null, req)); + asyncResponse(() -> { + GetFileInfoResponseProto res = asyncGet.get(-1, null); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + }); + return null; + } + GetFileInfoResponseProto res = ipc(() -> rpcProxy.getFileInfo(null, req)); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + } + + @Override + public HdfsLocatedFileStatus getLocatedFileInfo(String src, + boolean needBlockToken) throws IOException { + GetLocatedFileInfoRequestProto req = + GetLocatedFileInfoRequestProto.newBuilder() + .setSrc(src) + .setNeedBlockToken(needBlockToken) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getLocatedFileInfo(null, req)); + asyncResponse((AsyncRpcProtocolPBUtil.Response) () -> { + GetLocatedFileInfoResponseProto res = asyncGet.get(-1, null); + return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null; + }); + return null; + } + GetLocatedFileInfoResponseProto res = + ipc(() -> rpcProxy.getLocatedFileInfo(null, req)); + return (HdfsLocatedFileStatus) (res.hasFs() + ? PBHelperClient.convert(res.getFs()) + : null); + } + + @Override + public HdfsFileStatus getFileLinkInfo(String src) throws IOException { + GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder() + .setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getFileLinkInfo(null, req)); + asyncResponse(() -> { + GetFileLinkInfoResponseProto result = asyncGet.get(-1, null); + return result.hasFs() ? PBHelperClient.convert(result.getFs()) : null; + }); + return null; + } + GetFileLinkInfoResponseProto result = ipc(() -> rpcProxy.getFileLinkInfo(null, req)); + return result.hasFs() ? PBHelperClient.convert(result.getFs()) : null; + } + + @Override + public ContentSummary getContentSummary(String path) throws IOException { + GetContentSummaryRequestProto req = GetContentSummaryRequestProto + .newBuilder() + .setPath(path) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getContentSummary(null, req)); + asyncResponse(() -> PBHelperClient.convert( + asyncGet.get(-1, null).getSummary())); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getContentSummary(null, req)) + .getSummary()); + } + + @Override + public void setQuota(String path, long namespaceQuota, long storagespaceQuota, + StorageType type) throws IOException { + final SetQuotaRequestProto.Builder builder + = SetQuotaRequestProto.newBuilder() + .setPath(path) + .setNamespaceQuota(namespaceQuota) + .setStoragespaceQuota(storagespaceQuota); + if (type != null) { + builder.setStorageType(PBHelperClient.convertStorageType(type)); + } + final SetQuotaRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setQuota(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.setQuota(null, req)); + } + } + + @Override + public void fsync(String src, long fileId, String client, + long lastBlockLength) throws IOException { + FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src) + .setClient(client).setLastBlockLength(lastBlockLength) + .setFileId(fileId).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.fsync(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.fsync(null, req)); + } + } + + @Override + public void setTimes(String src, long mtime, long atime) throws IOException { + SetTimesRequestProto req = SetTimesRequestProto.newBuilder() + .setSrc(src) + .setMtime(mtime) + .setAtime(atime) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setTimes(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.setTimes(null, req)); + } + } + + @Override + public void createSymlink(String target, String link, FsPermission dirPerm, + boolean createParent) throws IOException { + CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder() + .setTarget(target) + .setLink(link) + .setDirPerm(PBHelperClient.convert(dirPerm)) + .setCreateParent(createParent) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.createSymlink(null, req)); + asyncResponse(() -> asyncGet.get(-1, null)); + } else { + ipc(() -> rpcProxy.createSymlink(null, req)); + } + } + + @Override + public String getLinkTarget(String path) throws IOException { + GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder() + .setPath(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getLinkTarget(null, req)); + asyncResponse(new AsyncRpcProtocolPBUtil.Response() { + @Override + public Object response() throws Exception { + GetLinkTargetResponseProto rsp = asyncGet.get(-1, null); + return rsp.hasTargetPath() ? rsp.getTargetPath() : null; + } + }); + return null; + } + GetLinkTargetResponseProto rsp = ipc(() -> rpcProxy.getLinkTarget(null, req)); + return rsp.hasTargetPath() ? rsp.getTargetPath() : null; + } + + @Override + public LocatedBlock updateBlockForPipeline(ExtendedBlock block, + String clientName) throws IOException { + UpdateBlockForPipelineRequestProto req = UpdateBlockForPipelineRequestProto + .newBuilder() + .setBlock(PBHelperClient.convert(block)) + .setClientName(clientName) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.updateBlockForPipeline(null, req)); + asyncResponse( + () -> PBHelperClient.convertLocatedBlockProto( + asyncGet.get(-1, null).getBlock())); + return null; + } + return PBHelperClient.convertLocatedBlockProto( + ipc(() -> rpcProxy.updateBlockForPipeline(null, req)).getBlock()); + } + + @Override + public void updatePipeline(String clientName, ExtendedBlock oldBlock, + ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) + throws IOException { + UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder() + .setClientName(clientName) + .setOldBlock(PBHelperClient.convert(oldBlock)) + .setNewBlock(PBHelperClient.convert(newBlock)) + .addAllNewNodes(Arrays.asList(PBHelperClient.convert(newNodes))) + .addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.updatePipeline(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.updatePipeline(null, req)); + } + } + + @Override + public Token getDelegationToken(Text renewer) + throws IOException { + GetDelegationTokenRequestProto req = GetDelegationTokenRequestProto + .newBuilder() + .setRenewer(renewer == null ? "" : renewer.toString()) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDelegationToken(null, req)); + asyncResponse(() -> { + GetDelegationTokenResponseProto resp = asyncGet.get(-1, null); + return resp.hasToken() ? + PBHelperClient.convertDelegationToken(resp.getToken()) : null; + }); + return null; + } + GetDelegationTokenResponseProto resp = + ipc(() -> rpcProxy.getDelegationToken(null, req)); + return resp.hasToken() ? + PBHelperClient.convertDelegationToken(resp.getToken()) : null; + } + + @Override + public long renewDelegationToken(Token token) + throws IOException { + RenewDelegationTokenRequestProto req = + RenewDelegationTokenRequestProto.newBuilder(). + setToken(PBHelperClient.convert(token)). + build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.renewDelegationToken(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getNewExpiryTime()); + return -1; + } + return ipc(() -> rpcProxy.renewDelegationToken(null, req)).getNewExpiryTime(); + } + + @Override + public void cancelDelegationToken(Token token) + throws IOException { + CancelDelegationTokenRequestProto req = CancelDelegationTokenRequestProto + .newBuilder() + .setToken(PBHelperClient.convert(token)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.cancelDelegationToken(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.cancelDelegationToken(null, req)); + } + } + + @Override + public void setBalancerBandwidth(long bandwidth) throws IOException { + SetBalancerBandwidthRequestProto req = + SetBalancerBandwidthRequestProto.newBuilder() + .setBandwidth(bandwidth) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setBalancerBandwidth(null, req)); + asyncResponse(new Response() { + @Override + public Object response() throws Exception { + asyncGet.get(-1, null); + return null; + } + }); + } else { + ipc(() -> rpcProxy.setBalancerBandwidth(null, req)); + } + } + + @Override + public boolean isMethodSupported(String methodName) throws IOException { + return RpcClientUtil.isMethodSupported(rpcProxy, + ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER, + RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName); + } + + @Override + public DataEncryptionKey getDataEncryptionKey() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getDataEncryptionKey(null, + VOID_GET_DATA_ENCRYPTIONKEY_REQUEST)); + asyncResponse(() -> { + GetDataEncryptionKeyResponseProto rsp = asyncGet.get(-1, null); + return rsp.hasDataEncryptionKey() ? + PBHelperClient.convert(rsp.getDataEncryptionKey()) : null; + }); + return null; + } + GetDataEncryptionKeyResponseProto rsp = ipc(() -> rpcProxy.getDataEncryptionKey( + null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST)); + return rsp.hasDataEncryptionKey() ? + PBHelperClient.convert(rsp.getDataEncryptionKey()) : null; + } + + + @Override + public boolean isFileClosed(String src) throws IOException { + IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder() + .setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.isFileClosed(null, req)); + asyncResponse(() -> asyncGet.get(-1, null).getResult()); + return true; + } + return ipc(() -> rpcProxy.isFileClosed(null, req)).getResult(); + } + + @Override + public Object getUnderlyingProxyObject() { + return rpcProxy; + } + + @Override + public String createSnapshot(String snapshotRoot, String snapshotName) + throws IOException { + final CreateSnapshotRequestProto.Builder builder + = CreateSnapshotRequestProto.newBuilder().setSnapshotRoot(snapshotRoot); + if (snapshotName != null) { + builder.setSnapshotName(snapshotName); + } + final CreateSnapshotRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.createSnapshot(null, req)); + asyncResponse(() -> + asyncGet.get(-1, null).getSnapshotPath()); + return null; + } + return ipc(() -> rpcProxy.createSnapshot(null, req)).getSnapshotPath(); + } + + @Override + public void deleteSnapshot(String snapshotRoot, String snapshotName) + throws IOException { + DeleteSnapshotRequestProto req = DeleteSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setSnapshotName(snapshotName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.deleteSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.deleteSnapshot(null, req)); + } + } + + @Override + public void allowSnapshot(String snapshotRoot) throws IOException { + AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.allowSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.allowSnapshot(null, req)); + } + } + + @Override + public void disallowSnapshot(String snapshotRoot) throws IOException { + DisallowSnapshotRequestProto req = DisallowSnapshotRequestProto + .newBuilder().setSnapshotRoot(snapshotRoot).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.disallowSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.disallowSnapshot(null, req)); + } + } + + @Override + public void renameSnapshot(String snapshotRoot, String snapshotOldName, + String snapshotNewName) throws IOException { + RenameSnapshotRequestProto req = RenameSnapshotRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setSnapshotOldName(snapshotOldName) + .setSnapshotNewName(snapshotNewName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.renameSnapshot(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.renameSnapshot(null, req)); + } + } + + @Override + public SnapshottableDirectoryStatus[] getSnapshottableDirListing() + throws IOException { + GetSnapshottableDirListingRequestProto req = + GetSnapshottableDirListingRequestProto.newBuilder().build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshottableDirListing(null, req)); + asyncResponse(() -> { + GetSnapshottableDirListingResponseProto result = asyncGet.get(-1, null); + if (result.hasSnapshottableDirList()) { + return PBHelperClient.convert(result.getSnapshottableDirList()); + } + return null; + }); + return null; + } + GetSnapshottableDirListingResponseProto result = ipc(() -> rpcProxy + .getSnapshottableDirListing(null, req)); + + if (result.hasSnapshottableDirList()) { + return PBHelperClient.convert(result.getSnapshottableDirList()); + } + return null; + } + + @Override + public SnapshotStatus[] getSnapshotListing(String path) + throws IOException { + GetSnapshotListingRequestProto req = + GetSnapshotListingRequestProto.newBuilder() + .setSnapshotRoot(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshotListing(null, req)); + asyncResponse(() -> { + GetSnapshotListingResponseProto result = asyncGet.get(-1, null); + if (result.hasSnapshotList()) { + return PBHelperClient.convert(result.getSnapshotList()); + } + return null; + }); + return null; + } + GetSnapshotListingResponseProto result = ipc(() -> rpcProxy + .getSnapshotListing(null, req)); + + if (result.hasSnapshotList()) { + return PBHelperClient.convert(result.getSnapshotList()); + } + return null; + } + + @Override + public SnapshotDiffReport getSnapshotDiffReport( + String snapshotRoot, + String fromSnapshot, String toSnapshot) throws IOException { + GetSnapshotDiffReportRequestProto req = GetSnapshotDiffReportRequestProto + .newBuilder().setSnapshotRoot(snapshotRoot) + .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshotDiffReport(null, req)); + asyncResponse(() -> PBHelperClient.convert(asyncGet.get(-1, null) + .getDiffReport())); + return null; + } + GetSnapshotDiffReportResponseProto result = + ipc(() -> rpcProxy.getSnapshotDiffReport(null, req)); + + return PBHelperClient.convert(result.getDiffReport()); + } + + @Override + public SnapshotDiffReportListing getSnapshotDiffReportListing( + String snapshotRoot, String fromSnapshot, String toSnapshot, + byte[] startPath, int index) throws IOException { + GetSnapshotDiffReportListingRequestProto req = + GetSnapshotDiffReportListingRequestProto.newBuilder() + .setSnapshotRoot(snapshotRoot).setFromSnapshot(fromSnapshot) + .setToSnapshot(toSnapshot).setCursor( + HdfsProtos.SnapshotDiffReportCursorProto.newBuilder() + .setStartPath(PBHelperClient.getByteString(startPath)) + .setIndex(index).build()).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSnapshotDiffReportListing(null, req)); + asyncResponse(new Response() { + @Override + public Object response() throws Exception { + return PBHelperClient.convert(asyncGet.get(-1, null).getDiffReport()); + } + }); + return null; + } + GetSnapshotDiffReportListingResponseProto result = + ipc(() -> rpcProxy.getSnapshotDiffReportListing(null, req)); + + return PBHelperClient.convert(result.getDiffReport()); + } + + @Override + public long addCacheDirective(CacheDirectiveInfo directive, + EnumSet flags) throws IOException { + AddCacheDirectiveRequestProto.Builder builder = + AddCacheDirectiveRequestProto.newBuilder(). + setInfo(PBHelperClient.convert(directive)); + if (!flags.isEmpty()) { + builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags)); + } + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.addCacheDirective(null, builder.build())); + asyncResponse(() -> asyncGet.get(-1, null).getId()); + return -1; + } + return ipc(() -> rpcProxy.addCacheDirective(null, builder.build())).getId(); + } + + @Override + public void modifyCacheDirective(CacheDirectiveInfo directive, + EnumSet flags) throws IOException { + ModifyCacheDirectiveRequestProto.Builder builder = + ModifyCacheDirectiveRequestProto.newBuilder(). + setInfo(PBHelperClient.convert(directive)); + if (!flags.isEmpty()) { + builder.setCacheFlags(PBHelperClient.convertCacheFlags(flags)); + } + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.modifyCacheDirective(null, builder.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.modifyCacheDirective(null, builder.build())); + } + } + + @Override + public void removeCacheDirective(long id) + throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeCacheDirective(null, + RemoveCacheDirectiveRequestProto.newBuilder(). + setId(id).build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeCacheDirective(null, + RemoveCacheDirectiveRequestProto.newBuilder(). + setId(id).build())); + } + } + + private static class BatchedCacheEntries + implements BatchedEntries { + private final ListCacheDirectivesResponseProto response; + + BatchedCacheEntries( + ListCacheDirectivesResponseProto response) { + this.response = response; + } + + @Override + public CacheDirectiveEntry get(int i) { + return PBHelperClient.convert(response.getElements(i)); + } + + @Override + public int size() { + return response.getElementsCount(); + } + + @Override + public boolean hasMore() { + return response.getHasMore(); + } + } + + @Override + public BatchedEntries listCacheDirectives( + long prevId, + CacheDirectiveInfo filter) throws IOException { + if (filter == null) { + filter = new CacheDirectiveInfo.Builder().build(); + } + CacheDirectiveInfo f = filter; + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.listCacheDirectives(null, + ListCacheDirectivesRequestProto.newBuilder(). + setPrevId(prevId). + setFilter(PBHelperClient.convert(f)). + build())); + asyncResponse(() -> new BatchedCacheEntries(asyncGet.get(-1, null))); + return null; + } + return new BatchedCacheEntries( + ipc(() -> rpcProxy.listCacheDirectives(null, + ListCacheDirectivesRequestProto.newBuilder(). + setPrevId(prevId). + setFilter(PBHelperClient.convert(f)). + build()))); + } + + @Override + public void addCachePool(CachePoolInfo info) throws IOException { + AddCachePoolRequestProto.Builder builder = + AddCachePoolRequestProto.newBuilder(); + builder.setInfo(PBHelperClient.convert(info)); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.addCachePool(null, builder.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.addCachePool(null, builder.build())); + } + } + + @Override + public void modifyCachePool(CachePoolInfo req) throws IOException { + ModifyCachePoolRequestProto.Builder builder = + ModifyCachePoolRequestProto.newBuilder(); + builder.setInfo(PBHelperClient.convert(req)); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.modifyCachePool(null, builder.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.modifyCachePool(null, builder.build())); + } + } + + @Override + public void removeCachePool(String cachePoolName) throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeCachePool(null, + RemoveCachePoolRequestProto.newBuilder(). + setPoolName(cachePoolName).build())); + + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeCachePool(null, + RemoveCachePoolRequestProto.newBuilder(). + setPoolName(cachePoolName).build())); + } + } + + private static class BatchedCachePoolEntries + implements BatchedEntries { + private final ListCachePoolsResponseProto proto; + + BatchedCachePoolEntries(ListCachePoolsResponseProto proto) { + this.proto = proto; + } + + @Override + public CachePoolEntry get(int i) { + CachePoolEntryProto elem = proto.getEntries(i); + return PBHelperClient.convert(elem); + } + + @Override + public int size() { + return proto.getEntriesCount(); + } + + @Override + public boolean hasMore() { + return proto.getHasMore(); + } + } + + @Override + public BatchedEntries listCachePools(String prevKey) + throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listCachePools(null, + ListCachePoolsRequestProto.newBuilder(). + setPrevPoolName(prevKey).build())); + asyncResponse((Response) () -> + new BatchedCachePoolEntries(asyncGet.get(-1, null))); + } + return new BatchedCachePoolEntries( + ipc(() -> rpcProxy.listCachePools(null, + ListCachePoolsRequestProto.newBuilder(). + setPrevPoolName(prevKey).build()))); + } + + @Override + public void modifyAclEntries(String src, List aclSpec) + throws IOException { + ModifyAclEntriesRequestProto req = ModifyAclEntriesRequestProto + .newBuilder().setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.modifyAclEntries(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.modifyAclEntries(null, req)); + } + } + + @Override + public void removeAclEntries(String src, List aclSpec) + throws IOException { + RemoveAclEntriesRequestProto req = RemoveAclEntriesRequestProto + .newBuilder().setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeAclEntries(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeAclEntries(null, req)); + } + } + + @Override + public void removeDefaultAcl(String src) throws IOException { + RemoveDefaultAclRequestProto req = RemoveDefaultAclRequestProto + .newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeDefaultAcl(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeDefaultAcl(null, req)); + } + } + + @Override + public void removeAcl(String src) throws IOException { + RemoveAclRequestProto req = RemoveAclRequestProto.newBuilder() + .setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeAcl(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeAcl(null, req)); + } + } + + @Override + public void setAcl(String src, List aclSpec) throws IOException { + SetAclRequestProto req = SetAclRequestProto.newBuilder() + .setSrc(src) + .addAllAclSpec(PBHelperClient.convertAclEntryProto(aclSpec)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setAcl(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setAcl(null, req)); + } + } + + @Override + public AclStatus getAclStatus(String src) throws IOException { + GetAclStatusRequestProto req = GetAclStatusRequestProto.newBuilder() + .setSrc(src).build(); + try { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getAclStatus(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } else { + return PBHelperClient.convert(rpcProxy.getAclStatus(null, req)); + } + } catch (ServiceException e) { + throw getRemoteException(e); + } + } + + @Override + public void createEncryptionZone(String src, String keyName) + throws IOException { + final CreateEncryptionZoneRequestProto.Builder builder = + CreateEncryptionZoneRequestProto.newBuilder(); + builder.setSrc(src); + if (keyName != null && !keyName.isEmpty()) { + builder.setKeyName(keyName); + } + CreateEncryptionZoneRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.createEncryptionZone(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.createEncryptionZone(null, req)); + } + } + + @Override + public EncryptionZone getEZForPath(String src) throws IOException { + final GetEZForPathRequestProto.Builder builder = + GetEZForPathRequestProto.newBuilder(); + builder.setSrc(src); + final GetEZForPathRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getEZForPath(null, req)); + asyncResponse((Response) () -> { + final EncryptionZonesProtos.GetEZForPathResponseProto response + = asyncGet.get(-1, null); + if (response.hasZone()) { + return PBHelperClient.convert(response.getZone()); + } else { + return null; + } + }); + return null; + } + final EncryptionZonesProtos.GetEZForPathResponseProto response = + ipc(() -> rpcProxy.getEZForPath(null, req)); + if (response.hasZone()) { + return PBHelperClient.convert(response.getZone()); + } else { + return null; + } + } + + @Override + public BatchedEntries listEncryptionZones(long id) + throws IOException { + final ListEncryptionZonesRequestProto req = + ListEncryptionZonesRequestProto.newBuilder() + .setId(id) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listEncryptionZones(null, req)); + asyncResponse((Response) () -> { + EncryptionZonesProtos.ListEncryptionZonesResponseProto response + = asyncGet.get(-1, null); + List elements = + Lists.newArrayListWithCapacity(response.getZonesCount()); + for (EncryptionZoneProto p : response.getZonesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + }); + return null; + } + EncryptionZonesProtos.ListEncryptionZonesResponseProto response = + ipc(() -> rpcProxy.listEncryptionZones(null, req)); + List elements = + Lists.newArrayListWithCapacity(response.getZonesCount()); + for (EncryptionZoneProto p : response.getZonesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + } + + @Override + public void setErasureCodingPolicy(String src, String ecPolicyName) + throws IOException { + final SetErasureCodingPolicyRequestProto.Builder builder = + SetErasureCodingPolicyRequestProto.newBuilder(); + builder.setSrc(src); + if (ecPolicyName != null) { + builder.setEcPolicyName(ecPolicyName); + } + SetErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setErasureCodingPolicy(null, req)); + } + } + + @Override + public void unsetErasureCodingPolicy(String src) throws IOException { + final UnsetErasureCodingPolicyRequestProto.Builder builder = + UnsetErasureCodingPolicyRequestProto.newBuilder(); + builder.setSrc(src); + UnsetErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.unsetErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.unsetErasureCodingPolicy(null, req)); + } + } + + @Override + public ECTopologyVerifierResult getECTopologyResultForPolicies( + final String... policyNames) throws IOException { + final GetECTopologyResultForPoliciesRequestProto.Builder builder = + GetECTopologyResultForPoliciesRequestProto.newBuilder(); + builder.addAllPolicies(Arrays.asList(policyNames)); + GetECTopologyResultForPoliciesRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getECTopologyResultForPolicies(null, req)); + asyncResponse((Response) () -> PBHelperClient + .convertECTopologyVerifierResultProto( + asyncGet.get(-1, null).getResponse())); + } + GetECTopologyResultForPoliciesResponseProto response = + ipc(() -> rpcProxy.getECTopologyResultForPolicies(null, req)); + return PBHelperClient + .convertECTopologyVerifierResultProto(response.getResponse()); + } + + @Override + public void reencryptEncryptionZone(String zone, ReencryptAction action) + throws IOException { + final ReencryptEncryptionZoneRequestProto.Builder builder = + ReencryptEncryptionZoneRequestProto.newBuilder(); + builder.setZone(zone).setAction(PBHelperClient.convert(action)); + ReencryptEncryptionZoneRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.reencryptEncryptionZone(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.reencryptEncryptionZone(null, req)); + } + } + + @Override + public BatchedEntries listReencryptionStatus(long id) + throws IOException { + final ListReencryptionStatusRequestProto req = + ListReencryptionStatusRequestProto.newBuilder().setId(id).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listReencryptionStatus(null, req)); + asyncResponse((Response) () -> { + ListReencryptionStatusResponseProto response = asyncGet.get(-1, null); + List elements = + Lists.newArrayListWithCapacity(response.getStatusesCount()); + for (ZoneReencryptionStatusProto p : response.getStatusesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + }); + return null; + } + ListReencryptionStatusResponseProto response = + ipc(() -> rpcProxy.listReencryptionStatus(null, req)); + List elements = + Lists.newArrayListWithCapacity(response.getStatusesCount()); + for (ZoneReencryptionStatusProto p : response.getStatusesList()) { + elements.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(elements, response.getHasMore()); + } + + @Override + public void setXAttr(String src, XAttr xAttr, EnumSet flag) + throws IOException { + SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder() + .setSrc(src) + .setXAttr(PBHelperClient.convertXAttrProto(xAttr)) + .setFlag(PBHelperClient.convert(flag)) + .build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setXAttr(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setXAttr(null, req)); + } + } + + @Override + public List getXAttrs(String src, List xAttrs) + throws IOException { + GetXAttrsRequestProto.Builder builder = GetXAttrsRequestProto.newBuilder(); + builder.setSrc(src); + if (xAttrs != null) { + builder.addAllXAttrs(PBHelperClient.convertXAttrProto(xAttrs)); + } + GetXAttrsRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getXAttrs(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getXAttrs(null, req))); + } + + @Override + public List listXAttrs(String src) throws IOException { + ListXAttrsRequestProto.Builder builder = + ListXAttrsRequestProto.newBuilder(); + builder.setSrc(src); + ListXAttrsRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listXAttrs(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.listXAttrs(null, req))); + } + + @Override + public void removeXAttr(String src, XAttr xAttr) throws IOException { + RemoveXAttrRequestProto req = RemoveXAttrRequestProto + .newBuilder().setSrc(src) + .setXAttr(PBHelperClient.convertXAttrProto(xAttr)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeXAttr(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeXAttr(null, req)); + } + } + + @Override + public void checkAccess(String path, FsAction mode) throws IOException { + CheckAccessRequestProto req = CheckAccessRequestProto.newBuilder() + .setPath(path).setMode(PBHelperClient.convert(mode)).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.checkAccess(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.checkAccess(null, req)); + } + } + + @Override + public void setStoragePolicy(String src, String policyName) + throws IOException { + SetStoragePolicyRequestProto req = SetStoragePolicyRequestProto + .newBuilder().setSrc(src).setPolicyName(policyName).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.setStoragePolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.setStoragePolicy(null, req)); + } + } + + @Override + public void unsetStoragePolicy(String src) throws IOException { + UnsetStoragePolicyRequestProto req = UnsetStoragePolicyRequestProto + .newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.unsetStoragePolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.unsetStoragePolicy(null, req)); + } + } + + @Override + public BlockStoragePolicy getStoragePolicy(String path) throws IOException { + GetStoragePolicyRequestProto request = GetStoragePolicyRequestProto + .newBuilder().setPath(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getStoragePolicy(null, request)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null).getStoragePolicy())); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getStoragePolicy(null, request)) + .getStoragePolicy()); + } + + @Override + public BlockStoragePolicy[] getStoragePolicies() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy + .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST)); + asyncResponse((Response) () -> PBHelperClient.convertStoragePolicies( + asyncGet.get(-1, null).getPoliciesList())); + return null; + } + GetStoragePoliciesResponseProto response = ipc(() -> rpcProxy + .getStoragePolicies(null, VOID_GET_STORAGE_POLICIES_REQUEST)); + return PBHelperClient.convertStoragePolicies(response.getPoliciesList()); + } + + public long getCurrentEditLogTxid() throws IOException { + GetCurrentEditLogTxidRequestProto req = GetCurrentEditLogTxidRequestProto + .getDefaultInstance(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getCurrentEditLogTxid(null, req)); + asyncResponse((Response) () -> asyncGet.get(-1, null).getTxid()); + return -1; + } + return ipc(() -> rpcProxy.getCurrentEditLogTxid(null, req)).getTxid(); + } + + @Override + public EventBatchList getEditsFromTxid(long txid) throws IOException { + GetEditsFromTxidRequestProto req = GetEditsFromTxidRequestProto.newBuilder() + .setTxid(txid).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getEditsFromTxid(null, req)); + asyncResponse((Response) () -> PBHelperClient.convert(asyncGet.get(-1, null))); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getEditsFromTxid(null, req))); + } + + @Override + public AddErasureCodingPolicyResponse[] addErasureCodingPolicies( + ErasureCodingPolicy[] policies) throws IOException { + List protos = Arrays.stream(policies) + .map(PBHelperClient::convertErasureCodingPolicy) + .collect(Collectors.toList()); + AddErasureCodingPoliciesRequestProto req = + AddErasureCodingPoliciesRequestProto.newBuilder() + .addAllEcPolicies(protos).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy + .addErasureCodingPolicies(null, req)); + asyncResponse((Response) () -> { + AddErasureCodingPoliciesResponseProto rep = asyncGet.get(-1, null); + AddErasureCodingPolicyResponse[] responses = + rep.getResponsesList().stream() + .map(PBHelperClient::convertAddErasureCodingPolicyResponse) + .toArray(AddErasureCodingPolicyResponse[]::new); + return responses; + }); + return null; + } + AddErasureCodingPoliciesResponseProto rep = ipc(() -> rpcProxy + .addErasureCodingPolicies(null, req)); + AddErasureCodingPolicyResponse[] responses = + rep.getResponsesList().stream() + .map(PBHelperClient::convertAddErasureCodingPolicyResponse) + .toArray(AddErasureCodingPolicyResponse[]::new); + return responses; + } + + @Override + public void removeErasureCodingPolicy(String ecPolicyName) + throws IOException { + RemoveErasureCodingPolicyRequestProto.Builder builder = + RemoveErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + RemoveErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.removeErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.removeErasureCodingPolicy(null, req)); + } + } + + @Override + public void enableErasureCodingPolicy(String ecPolicyName) + throws IOException { + EnableErasureCodingPolicyRequestProto.Builder builder = + EnableErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + EnableErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.enableErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.enableErasureCodingPolicy(null, req)); + } + } + + @Override + public void disableErasureCodingPolicy(String ecPolicyName) + throws IOException { + DisableErasureCodingPolicyRequestProto.Builder builder = + DisableErasureCodingPolicyRequestProto.newBuilder(); + builder.setEcPolicyName(ecPolicyName); + DisableErasureCodingPolicyRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.disableErasureCodingPolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.disableErasureCodingPolicy(null, req)); + } + } + + @Override + public ErasureCodingPolicyInfo[] getErasureCodingPolicies() + throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy + .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST)); + asyncResponse((Response) () -> { + GetErasureCodingPoliciesResponseProto response = asyncGet.get(-1, null); + ErasureCodingPolicyInfo[] ecPolicies = + new ErasureCodingPolicyInfo[response.getEcPoliciesCount()]; + int i = 0; + for (ErasureCodingPolicyProto proto : response.getEcPoliciesList()) { + ecPolicies[i++] = + PBHelperClient.convertErasureCodingPolicyInfo(proto); + } + return ecPolicies; + }); + return null; + } + GetErasureCodingPoliciesResponseProto response = ipc(() -> rpcProxy + .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST)); + ErasureCodingPolicyInfo[] ecPolicies = + new ErasureCodingPolicyInfo[response.getEcPoliciesCount()]; + int i = 0; + for (ErasureCodingPolicyProto proto : response.getEcPoliciesList()) { + ecPolicies[i++] = + PBHelperClient.convertErasureCodingPolicyInfo(proto); + } + return ecPolicies; + } + + @Override + public Map getErasureCodingCodecs() throws IOException { + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy + .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST)); + asyncResponse(() -> { + GetErasureCodingCodecsResponseProto response = asyncGet.get(-1, null); + Map ecCodecs = new HashMap<>(); + for (CodecProto codec : response.getCodecList()) { + ecCodecs.put(codec.getCodec(), codec.getCoders()); + } + return ecCodecs; + }); + return null; + } + GetErasureCodingCodecsResponseProto response = ipc(() -> rpcProxy + .getErasureCodingCodecs(null, VOID_GET_EC_CODEC_REQUEST)); + Map ecCodecs = new HashMap<>(); + for (CodecProto codec : response.getCodecList()) { + ecCodecs.put(codec.getCodec(), codec.getCoders()); + } + return ecCodecs; + } + + @Override + public ErasureCodingPolicy getErasureCodingPolicy(String src) + throws IOException { + GetErasureCodingPolicyRequestProto req = + GetErasureCodingPolicyRequestProto.newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getErasureCodingPolicy(null, req)); + asyncResponse((Response) () -> { + GetErasureCodingPolicyResponseProto response = asyncGet.get(-1, null); + if (response.hasEcPolicy()) { + return PBHelperClient.convertErasureCodingPolicy( + response.getEcPolicy()); + } + return null; + }); + return null; + } + GetErasureCodingPolicyResponseProto response = + ipc(() -> rpcProxy.getErasureCodingPolicy(null, req)); + if (response.hasEcPolicy()) { + return PBHelperClient.convertErasureCodingPolicy( + response.getEcPolicy()); + } + return null; + } + + @Override + public QuotaUsage getQuotaUsage(String path) throws IOException { + GetQuotaUsageRequestProto req = + GetQuotaUsageRequestProto.newBuilder().setPath(path).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getQuotaUsage(null, req)); + asyncResponse((Response) () -> + PBHelperClient.convert(asyncGet.get(-1, null).getUsage())); + return null; + } + return PBHelperClient.convert(ipc(() -> rpcProxy.getQuotaUsage(null, req)) + .getUsage()); + } + + @Deprecated + @Override + public BatchedEntries listOpenFiles(long prevId) + throws IOException { + return listOpenFiles(prevId, EnumSet.of(OpenFilesType.ALL_OPEN_FILES), + OpenFilesIterator.FILTER_PATH_DEFAULT); + } + + @Override + public BatchedEntries listOpenFiles( + long prevId, + EnumSet openFilesTypes, String path) throws IOException { + ListOpenFilesRequestProto.Builder req = + ListOpenFilesRequestProto.newBuilder().setId(prevId); + if (openFilesTypes != null) { + req.addAllTypes(PBHelperClient.convertOpenFileTypes(openFilesTypes)); + } + req.setPath(path); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.listOpenFiles(null, req.build())); + asyncResponse(() -> { + ListOpenFilesResponseProto response = asyncGet.get(-1, null); + List openFileEntries = + Lists.newArrayListWithCapacity(response.getEntriesCount()); + for (OpenFilesBatchResponseProto p : response.getEntriesList()) { + openFileEntries.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(openFileEntries, response.getHasMore()); + }); + return null; + } + ListOpenFilesResponseProto response = + ipc(() -> rpcProxy.listOpenFiles(null, req.build())); + List openFileEntries = + Lists.newArrayListWithCapacity(response.getEntriesCount()); + for (OpenFilesBatchResponseProto p : response.getEntriesList()) { + openFileEntries.add(PBHelperClient.convert(p)); + } + return new BatchedListEntries<>(openFileEntries, response.getHasMore()); + } + + @Override + public void msync() throws IOException { + MsyncRequestProto.Builder req = MsyncRequestProto.newBuilder(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.msync(null, req.build())); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.msync(null, req.build())); + } + } + + @Override + public void satisfyStoragePolicy(String src) throws IOException { + SatisfyStoragePolicyRequestProto req = + SatisfyStoragePolicyRequestProto.newBuilder().setSrc(src).build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.satisfyStoragePolicy(null, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } else { + ipc(() -> rpcProxy.satisfyStoragePolicy(null, req)); + } + } + + @Override + public DatanodeInfo[] getSlowDatanodeReport() throws IOException { + GetSlowDatanodeReportRequestProto req = + GetSlowDatanodeReportRequestProto.newBuilder().build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getSlowDatanodeReport(null, req)); + asyncResponse(() -> + PBHelperClient.convert(asyncGet.get(-1, null).getDatanodeInfoProtoList())); + return null; + } + return PBHelperClient.convert( + ipc(() -> rpcProxy.getSlowDatanodeReport(null, req)).getDatanodeInfoProtoList()); + } + + @Override + public HAServiceProtocol.HAServiceState getHAServiceState() + throws IOException { + HAServiceStateRequestProto req = + HAServiceStateRequestProto.newBuilder().build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getHAServiceState(null, req)); + asyncResponse(() -> { + HAServiceStateProto res = asyncGet.get(-1, null).getState(); + switch(res) { + case ACTIVE: + return HAServiceProtocol.HAServiceState.ACTIVE; + case STANDBY: + return HAServiceProtocol.HAServiceState.STANDBY; + case OBSERVER: + return HAServiceProtocol.HAServiceState.OBSERVER; + case INITIALIZING: + default: + return HAServiceProtocol.HAServiceState.INITIALIZING; + } + }); + return null; + } + HAServiceStateProto res = + ipc(() -> rpcProxy.getHAServiceState(null, req)).getState(); + switch(res) { + case ACTIVE: + return HAServiceProtocol.HAServiceState.ACTIVE; + case STANDBY: + return HAServiceProtocol.HAServiceState.STANDBY; + case OBSERVER: + return HAServiceProtocol.HAServiceState.OBSERVER; + case INITIALIZING: + default: + return HAServiceProtocol.HAServiceState.INITIALIZING; + } + } + + @Override + public Path getEnclosingRoot(String filename) throws IOException { + final GetEnclosingRootRequestProto.Builder builder = + GetEnclosingRootRequestProto.newBuilder(); + builder.setFilename(filename); + final GetEnclosingRootRequestProto req = builder.build(); + + if (Client.isAsynchronousMode()) { + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.getEnclosingRoot(null, req)); + asyncResponse(() -> + new Path(asyncGet.get(-1, null).getEnclosingRootPath())); + return null; + } + try { + final GetEnclosingRootResponseProto response = + rpcProxy.getEnclosingRoot(null, req); + return new Path(response.getEnclosingRootPath()); + } catch (ServiceException e) { + throw getRemoteException(e); + } + } +} \ No newline at end of file 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..94856dcaf7a01 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterGetUserMappingsProtocolTranslatorPB.java @@ -0,0 +1,57 @@ +/** + * 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 org.apache.hadoop.util.concurrent.AsyncGet; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; + +public class RouterGetUserMappingsProtocolTranslatorPB + extends GetUserMappingsProtocolClientSideTranslatorPB { + private final GetUserMappingsProtocolPB rpcProxy; + + public RouterGetUserMappingsProtocolTranslatorPB(GetUserMappingsProtocolPB rpcProxy) { + super(rpcProxy); + this.rpcProxy = rpcProxy; + } + + @Override + public String[] getGroupsForUser(String user) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getGroupsForUser(user); + } + GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto request = + GetUserMappingsProtocolProtos.GetGroupsForUserRequestProto + .newBuilder().setUser(user).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getGroupsForUser(NULL_CONTROLLER, request)); + asyncResponse(() -> { + GetUserMappingsProtocolProtos.GetGroupsForUserResponseProto resp = asyncGet.get(-1, null); + return resp.getGroupsList().toArray(new String[resp.getGroupsCount()]); + }); + return null; + } +} \ No newline at end of file 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..2e193db2146cf --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterNamenodeProtocolTranslatorPB.java @@ -0,0 +1,283 @@ +/** + * 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; +import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys; +import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; +import org.apache.hadoop.hdfs.server.namenode.NNStorage; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; +import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand; +import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; +import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; +import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; +import org.apache.hadoop.ipc.Client; +import org.apache.hadoop.util.concurrent.AsyncGet; +import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.GetMostRecentNameNodeFileTxIdResponseProto; + + +import java.io.IOException; +import org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.Response; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; + +public class RouterNamenodeProtocolTranslatorPB extends NamenodeProtocolTranslatorPB{ + private final NamenodeProtocolPB rpcProxy; + + public RouterNamenodeProtocolTranslatorPB(NamenodeProtocolPB rpcProxy) { + super(rpcProxy); + this.rpcProxy = rpcProxy; + } + + @Override + public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long + minBlockSize, long timeInterval, StorageType storageType) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBlocks(datanode, size, minBlockSize, timeInterval, storageType); + } + NamenodeProtocolProtos.GetBlocksRequestProto.Builder builder = + NamenodeProtocolProtos.GetBlocksRequestProto.newBuilder() + .setDatanode(PBHelperClient.convert((DatanodeID)datanode)).setSize(size) + .setMinBlockSize(minBlockSize).setTimeInterval(timeInterval); + if (storageType != null) { + builder.setStorageType(PBHelperClient.convertStorageType(storageType)); + } + NamenodeProtocolProtos.GetBlocksRequestProto req = builder.build(); + + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getBlocks(NULL_CONTROLLER, req)); + asyncResponse(() -> PBHelper.convert( + asyncGet.get(-1, null).getBlocks())); + return null; + } + + @Override + public ExportedBlockKeys getBlockKeys() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getBlockKeys(); + } + AsyncGet asyncGet = + asyncIpc(() -> + rpcProxy.getBlockKeys(NULL_CONTROLLER, VOID_GET_BLOCKKEYS_REQUEST)); + asyncResponse(() -> { + NamenodeProtocolProtos.GetBlockKeysResponseProto rsp = + asyncGet.get(-1, null); + return rsp.hasKeys() ? PBHelper.convert(rsp.getKeys()) : null; + }); + return null; + } + + @Override + public long getTransactionID() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getTransactionID(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getTransactionId(NULL_CONTROLLER, + VOID_GET_TRANSACTIONID_REQUEST)); + asyncResponse(() -> asyncGet.get(-1, null).getTxId()); + return -1; + } + + @Override + public long getMostRecentCheckpointTxId() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getMostRecentCheckpointTxId(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getMostRecentCheckpointTxId(NULL_CONTROLLER, + NamenodeProtocolProtos.GetMostRecentCheckpointTxIdRequestProto.getDefaultInstance())); + asyncResponse((Response) () -> asyncGet.get(-1, null).getTxId()); + return -1; + } + + @Override + public long getMostRecentNameNodeFileTxId(NNStorage.NameNodeFile nnf) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getMostRecentNameNodeFileTxId(nnf); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getMostRecentNameNodeFileTxId(NULL_CONTROLLER, + NamenodeProtocolProtos.GetMostRecentNameNodeFileTxIdRequestProto.newBuilder() + .setNameNodeFile(nnf.toString()).build())); + asyncResponse(() -> asyncGet.get(-1, null).getTxId()); + return -1; + } + + @Override + public CheckpointSignature rollEditLog() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.rollEditLog(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.rollEditLog(NULL_CONTROLLER, + VOID_ROLL_EDITLOG_REQUEST)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getSignature())); + return null; + } + + @Override + public NamespaceInfo versionRequest() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.versionRequest(); + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.versionRequest(NULL_CONTROLLER, + VOID_VERSION_REQUEST)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getInfo())); + return null; + } + + @Override + public void errorReport(NamenodeRegistration registration, int errorCode, + String msg) throws IOException { + if (!Client.isAsynchronousMode()) { + super.errorReport(registration, errorCode, msg); + return; + } + NamenodeProtocolProtos.ErrorReportRequestProto req = + NamenodeProtocolProtos.ErrorReportRequestProto.newBuilder() + .setErrorCode(errorCode).setMsg(msg) + .setRegistration(PBHelper.convert(registration)).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.errorReport(NULL_CONTROLLER, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public NamenodeRegistration registerSubordinateNamenode( + NamenodeRegistration registration) throws IOException { + if (!Client.isAsynchronousMode()) { + return super.registerSubordinateNamenode(registration); + } + NamenodeProtocolProtos.RegisterRequestProto req = + NamenodeProtocolProtos.RegisterRequestProto.newBuilder() + .setRegistration(PBHelper.convert(registration)).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.registerSubordinateNamenode(NULL_CONTROLLER, req)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getRegistration())); + return null; + } + + @Override + public NamenodeCommand startCheckpoint(NamenodeRegistration registration) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.startCheckpoint(registration); + } + NamenodeProtocolProtos.StartCheckpointRequestProto req = + NamenodeProtocolProtos.StartCheckpointRequestProto.newBuilder() + .setRegistration(PBHelper.convert(registration)).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.startCheckpoint(NULL_CONTROLLER, req)); + asyncResponse(() -> { + HdfsServerProtos.NamenodeCommandProto cmd = + asyncGet.get(-1, null).getCommand(); + return PBHelper.convert(cmd); + }); + return null; + } + + @Override + public void endCheckpoint(NamenodeRegistration registration, + CheckpointSignature sig) throws IOException { + if (!Client.isAsynchronousMode()) { + super.endCheckpoint(registration, sig); + return; + } + NamenodeProtocolProtos.EndCheckpointRequestProto req = + NamenodeProtocolProtos.EndCheckpointRequestProto.newBuilder() + .setRegistration(PBHelper.convert(registration)) + .setSignature(PBHelper.convert(sig)).build(); + AsyncGet asyncGet + = asyncIpc(() -> rpcProxy.endCheckpoint(NULL_CONTROLLER, req)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public RemoteEditLogManifest getEditLogManifest(long sinceTxId) + throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getEditLogManifest(sinceTxId); + } + NamenodeProtocolProtos.GetEditLogManifestRequestProto req = + NamenodeProtocolProtos.GetEditLogManifestRequestProto + .newBuilder().setSinceTxId(sinceTxId).build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.getEditLogManifest(NULL_CONTROLLER, req)); + asyncResponse(() -> PBHelper.convert(asyncGet.get(-1, null).getManifest())); + return null; + } + + @Override + public boolean isUpgradeFinalized() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.isUpgradeFinalized(); + } + NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto req = + NamenodeProtocolProtos.IsUpgradeFinalizedRequestProto + .newBuilder().build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.isUpgradeFinalized(NULL_CONTROLLER, req)); + asyncResponse(() -> asyncGet.get(-1, null).getIsUpgradeFinalized()); + return false; + } + + @Override + public boolean isRollingUpgrade() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.isRollingUpgrade(); + } + NamenodeProtocolProtos.IsRollingUpgradeRequestProto req = + NamenodeProtocolProtos.IsRollingUpgradeRequestProto + .newBuilder().build(); + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.isRollingUpgrade(NULL_CONTROLLER, req)); + asyncResponse(() -> asyncGet.get(-1, null).getIsRollingUpgrade()); + return false; + } + + @Override + public Long getNextSPSPath() throws IOException { + if (!Client.isAsynchronousMode()) { + return super.getNextSPSPath(); + } + NamenodeProtocolProtos.GetNextSPSPathRequestProto req = + NamenodeProtocolProtos.GetNextSPSPathRequestProto.newBuilder().build(); + AsyncGet ayncGet = + asyncIpc(() -> rpcProxy.getNextSPSPath(NULL_CONTROLLER, req)); + asyncResponse(() -> { + NamenodeProtocolProtos.GetNextSPSPathResponseProto nextSPSPath = + ayncGet.get(-1, null); + return nextSPSPath.hasSpsPath() ? nextSPSPath.getSpsPath() : null; + }); + return null; + } +} \ No newline at end of file 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..6130682bcef95 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterRefreshUserMappingsProtocolTranslatorPB.java @@ -0,0 +1,72 @@ +/** + * 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.protocolPB.RefreshUserMappingsProtocolClientSideTranslatorPB; +import org.apache.hadoop.security.protocolPB.RefreshUserMappingsProtocolPB; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsResponseProto; +import org.apache.hadoop.security.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationResponseProto; + +import org.apache.hadoop.util.concurrent.AsyncGet; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncIpc; +import static org.apache.hadoop.hdfs.protocolPB.AsyncRpcProtocolPBUtil.asyncResponse; + + +public class RouterRefreshUserMappingsProtocolTranslatorPB + extends RefreshUserMappingsProtocolClientSideTranslatorPB { + private final RefreshUserMappingsProtocolPB rpcProxy; + public RouterRefreshUserMappingsProtocolTranslatorPB(RefreshUserMappingsProtocolPB rpcProxy) { + super(rpcProxy); + this.rpcProxy = rpcProxy; + } + + @Override + public void refreshUserToGroupsMappings() throws IOException { + if (!Client.isAsynchronousMode()) { + super.refreshUserToGroupsMappings(); + return; + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.refreshUserToGroupsMappings(NULL_CONTROLLER, + VOID_REFRESH_USER_TO_GROUPS_MAPPING_REQUEST)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } + + @Override + public void refreshSuperUserGroupsConfiguration() throws IOException { + if (!Client.isAsynchronousMode()) { + super.refreshSuperUserGroupsConfiguration(); + return; + } + AsyncGet asyncGet = + asyncIpc(() -> rpcProxy.refreshSuperUserGroupsConfiguration(NULL_CONTROLLER, + VOID_REFRESH_SUPERUSER_GROUPS_CONFIGURATION_REQUEST)); + asyncResponse(() -> { + asyncGet.get(-1, null); + return null; + }); + } +} \ 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/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/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index d25e5ae4d3012..cdc0c748bf53f 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 @@ -73,11 +73,13 @@ 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.async.AsyncUtil; import org.apache.hadoop.hdfs.server.namenode.ha.ReadOnly; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.io.retry.RetryPolicy.RetryAction.RetryDecision; import org.apache.hadoop.ipc.CallerContext; +import org.apache.hadoop.ipc.Client; import org.apache.hadoop.ipc.ObserverRetryOnActiveException; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RetriableException; @@ -719,7 +721,8 @@ private Object invoke( int retryCount, final Method method, final Object obj, final Object... params) throws IOException { try { - return method.invoke(obj, params); + // TODO: only test, not commit! + return invoke(method, obj, params); } catch (IllegalAccessException | IllegalArgumentException e) { LOG.error("Unexpected exception while proxying API", e); return null; @@ -755,6 +758,21 @@ private Object invoke( } } + // TODO: only test, not commit!!! + private Object invoke( + final Method method, final Object obj, final Object... params) + throws InvocationTargetException, IllegalAccessException { + Client.setAsynchronousMode(true); + method.invoke(obj, params); + try { + return AsyncUtil.syncReturn(Object.class); + } catch (Exception e) { + throw new InvocationTargetException(e); + } finally { + Client.setAsynchronousMode(false); + } + } + /** * Check if the exception comes from an unavailable subcluster. * @param ioe IOException to check. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java index 40d0395721be4..a1b40e4c7d9c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/NamenodeProtocolTranslatorPB.java @@ -72,18 +72,18 @@ public class NamenodeProtocolTranslatorPB implements NamenodeProtocol, ProtocolMetaInterface, Closeable, ProtocolTranslator { /** RpcController is not used and hence is set to null */ - private final static RpcController NULL_CONTROLLER = null; + protected final static RpcController NULL_CONTROLLER = null; /* * Protobuf requests with no parameters instantiated only once */ - private static final GetBlockKeysRequestProto VOID_GET_BLOCKKEYS_REQUEST = + protected static final GetBlockKeysRequestProto VOID_GET_BLOCKKEYS_REQUEST = GetBlockKeysRequestProto.newBuilder().build(); - private static final GetTransactionIdRequestProto VOID_GET_TRANSACTIONID_REQUEST = + protected static final GetTransactionIdRequestProto VOID_GET_TRANSACTIONID_REQUEST = GetTransactionIdRequestProto.newBuilder().build(); - private static final RollEditLogRequestProto VOID_ROLL_EDITLOG_REQUEST = + protected static final RollEditLogRequestProto VOID_ROLL_EDITLOG_REQUEST = RollEditLogRequestProto.newBuilder().build(); - private static final VersionRequestProto VOID_VERSION_REQUEST = + protected static final VersionRequestProto VOID_VERSION_REQUEST = VersionRequestProto.newBuilder().build(); final private NamenodeProtocolPB rpcProxy; From 518b1099e0077b58fdb0b519396ce72fceba8be3 Mon Sep 17 00:00:00 2001 From: Jian Zhang <1361320460@qq.com> Date: Thu, 6 Jun 2024 22:32:15 +0800 Subject: [PATCH 4/4] async util --- .../hadoop/hdfs/server/federation/router/async/AsyncUtil.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java index 4bed7eb58dd8f..685219c39f641 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncUtil.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.Iterator; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.function.Function; @@ -120,7 +121,7 @@ public static void asyncComplete(R value) { */ public static void asyncThrowException(Throwable e) { CompletableFuture result = new CompletableFuture<>(); - result.completeExceptionally(e); + result.completeExceptionally(new CompletionException(e)); CUR_COMPLETABLE_FUTURE.set(result); }