[ https://issues.apache.org/jira/browse/HDFS-17543?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17859607#comment-17859607 ]
ASF GitHub Bot commented on HDFS-17543: --------------------------------------- KeeProMise commented on code in PR #6868: URL: https://github.com/apache/hadoop/pull/6868#discussion_r1650557408 ########## hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/AsyncForEachRun.java: ########## @@ -0,0 +1,185 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.federation.router.async; + +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; + +/** + * The AsyncForEachRun class is part of the asynchronous operation utilities + * within the Hadoop Distributed File System (HDFS) Federation router. + * It provides the functionality to perform asynchronous operations on each + * element of an Iterator, applying a given async function and then applying + * a final transformation function to the results. + * + * <p>This class is designed to work with other asynchronous interfaces and + * utility classes to enable complex asynchronous workflows. It allows for + * non-blocking execution of tasks, which can improve the performance and + * responsiveness of HDFS operations.</p> + * + * <p>The class implements the AsyncRun interface, which means it can be used + * in asynchronous task chains. It maintains an Iterator of elements to + * process, an asyncFunction to apply to each element, and a final + * transformation function (thenApply) to produce the final result.</p> + * + * <p>The run method initiates the asynchronous operation, and the doOnce + * method recursively applies the asyncFunction to each element and handles + * the results. If the satisfy flag is set, the operation is completed + * with the current result.</p> + * + * <p>AsyncForEachRun is used to implement the following semantics:</p> + * <pre> + * {@code + * for (I element : elements) { + * T res = asyncFunction(element); + * R result = thenApply(element, res); + * if (satisfyCondition(res, result)) { + * break; + * } + * } + * return result; + * } + * </pre> + * + * @param <I> the type of the elements being iterated over + * @param <T> the type of the intermediate result from the asyncFunction + * @param <R> the type of the final result after applying the thenApply function + * @see AsyncRun + * @see AsyncApplyFunction + * @see BiFunction + */ +public class AsyncForEachRun<I, T, R> implements AsyncRun<R> { + + private boolean satisfy = false; + private Iterator<I> iterator; + private I now; + private final CompletableFuture<R> result = new CompletableFuture<>(); + private AsyncApplyFunction<I, T> asyncFunction; + private BiFunction<AsyncForEachRun<I, T, R>, T, R> thenApply; Review Comment: @hfutatzhanghb Hi, I think one thenApply is enough. I personally think that we need to implement basic type functions, then complex types of business functions can be obtained by combining basic types. If the business code requires multiple thenApply, it can also be combined with AsyncUtil. > [ARR] AsyncUtil makes asynchronous code more concise and easier. > ---------------------------------------------------------------- > > Key: HDFS-17543 > URL: https://issues.apache.org/jira/browse/HDFS-17543 > Project: Hadoop HDFS > Issue Type: Sub-task > Reporter: Jian Zhang > Assignee: Jian Zhang > Priority: Major > Labels: pull-request-available > Attachments: HDFS-17543.001.patch > > > *Describe* > Using the original Java CompletableFuture to implement an asynchronous router > is not conducive to the development of the open source community due to poor > code readability and maintainability. Therefore, I have implemented a > lightweight tool that encapsulates CompletableFuture. > By using this AsyncUtil, it is easy to write readable asynchronous code, > which is easy for everyone to understand. I have provided an example of this > tool class in UT, demonstrating how to change synchronous methods to > asynchronous code. > > *Examples and tests* > SyncClass provides some common synchronization methods > AsyncClass is the corresponding asynchronous implementation > Use TestAsyncUtil to simultaneously test the methods corresponding to > SyncClass and AsyncClass, ensuring that both asynchronous and synchronous > methods return the same results > > {*}NOTE{*}: In HDFS-17545, I used this tool to implement RouterAsyncRpcClient > which extends RouterRpcClient. You can also view RouterAsyncRpcClient to > understand the use of this tool. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org