Skip to content

Commit

Permalink
[client] Introduce AdminApiWorkflow for workflows which lookup broker…
Browse files Browse the repository at this point in the history
… first then send request.
  • Loading branch information
loserwang1024 committed Mar 6, 2025
1 parent ba36d05 commit 439987e
Show file tree
Hide file tree
Showing 12 changed files with 991 additions and 75 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,10 @@

package com.alibaba.fluss.client.admin;

import com.alibaba.fluss.client.admin.internal.AdminApiFuture;
import com.alibaba.fluss.client.admin.internal.AdminApiHandler;
import com.alibaba.fluss.client.admin.internal.AdminApiWorkflow;
import com.alibaba.fluss.client.admin.internal.ListOffsetsHandler;
import com.alibaba.fluss.client.metadata.KvSnapshotMetadata;
import com.alibaba.fluss.client.metadata.KvSnapshots;
import com.alibaba.fluss.client.metadata.LakeSnapshot;
Expand All @@ -37,7 +41,6 @@
import com.alibaba.fluss.rpc.GatewayClientProxy;
import com.alibaba.fluss.rpc.RpcClient;
import com.alibaba.fluss.rpc.gateway.AdminGateway;
import com.alibaba.fluss.rpc.gateway.TabletServerGateway;
import com.alibaba.fluss.rpc.messages.CreateDatabaseRequest;
import com.alibaba.fluss.rpc.messages.CreateTableRequest;
import com.alibaba.fluss.rpc.messages.DatabaseExistsRequest;
Expand All @@ -52,29 +55,20 @@
import com.alibaba.fluss.rpc.messages.GetTableSchemaRequest;
import com.alibaba.fluss.rpc.messages.ListDatabasesRequest;
import com.alibaba.fluss.rpc.messages.ListDatabasesResponse;
import com.alibaba.fluss.rpc.messages.ListOffsetsRequest;
import com.alibaba.fluss.rpc.messages.ListPartitionInfosRequest;
import com.alibaba.fluss.rpc.messages.ListTablesRequest;
import com.alibaba.fluss.rpc.messages.ListTablesResponse;
import com.alibaba.fluss.rpc.messages.PbListOffsetsRespForBucket;
import com.alibaba.fluss.rpc.messages.TableExistsRequest;
import com.alibaba.fluss.rpc.messages.TableExistsResponse;
import com.alibaba.fluss.rpc.protocol.ApiError;
import com.alibaba.fluss.utils.MapUtils;

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;

import static com.alibaba.fluss.client.utils.ClientRpcMessageUtils.makeCreatePartitionRequest;
import static com.alibaba.fluss.client.utils.ClientRpcMessageUtils.makeDropPartitionRequest;
import static com.alibaba.fluss.client.utils.ClientRpcMessageUtils.makeListOffsetsRequest;
import static com.alibaba.fluss.client.utils.MetadataUtils.sendMetadataRequestAndRebuildCluster;
import static com.alibaba.fluss.utils.Preconditions.checkNotNull;

Expand Down Expand Up @@ -346,74 +340,18 @@ public ListOffsetsResult listOffsets(
metadataUpdater.checkAndUpdatePartitionMetadata(physicalTablePath);
partitionId = metadataUpdater.getPartitionIdOrElseThrow(physicalTablePath);
}
Map<Integer, ListOffsetsRequest> requestMap =
prepareListOffsetsRequests(
metadataUpdater, tableId, partitionId, buckets, offsetSpec);
Map<Integer, CompletableFuture<Long>> bucketToOffsetMap = MapUtils.newConcurrentHashMap();
for (int bucket : buckets) {
bucketToOffsetMap.put(bucket, new CompletableFuture<>());
}

sendListOffsetsRequest(metadataUpdater, client, requestMap, bucketToOffsetMap);
return new ListOffsetsResult(bucketToOffsetMap);
AdminApiFuture.SimpleAdminApiFuture<Integer, Long> future =
ListOffsetsHandler.newFuture(buckets);
AdminApiHandler<Integer, Long> handler =
new ListOffsetsHandler(metadataUpdater, tableId, partitionId, offsetSpec, client);
AdminApiWorkflow<Integer, Long> adminApiDriver = new AdminApiWorkflow<>(future, handler);
CompletableFuture.runAsync(adminApiDriver::maybeSendRequests);
return new ListOffsetsResult(future.all());
}

@Override
public void close() {
// nothing to do yet
}

private static Map<Integer, ListOffsetsRequest> prepareListOffsetsRequests(
MetadataUpdater metadataUpdater,
long tableId,
@Nullable Long partitionId,
Collection<Integer> buckets,
OffsetSpec offsetSpec) {
Map<Integer, List<Integer>> nodeForBucketList = new HashMap<>();
for (Integer bucketId : buckets) {
int leader = metadataUpdater.leaderFor(new TableBucket(tableId, partitionId, bucketId));
nodeForBucketList.computeIfAbsent(leader, k -> new ArrayList<>()).add(bucketId);
}

Map<Integer, ListOffsetsRequest> listOffsetsRequests = new HashMap<>();
nodeForBucketList.forEach(
(leader, ids) ->
listOffsetsRequests.put(
leader,
makeListOffsetsRequest(tableId, partitionId, ids, offsetSpec)));
return listOffsetsRequests;
}

private static void sendListOffsetsRequest(
MetadataUpdater metadataUpdater,
RpcClient client,
Map<Integer, ListOffsetsRequest> leaderToRequestMap,
Map<Integer, CompletableFuture<Long>> bucketToOffsetMap) {
leaderToRequestMap.forEach(
(leader, request) -> {
TabletServerGateway gateway =
GatewayClientProxy.createGatewayProxy(
() -> metadataUpdater.getTabletServer(leader),
client,
TabletServerGateway.class);
gateway.listOffsets(request)
.thenAccept(
r -> {
for (PbListOffsetsRespForBucket resp :
r.getBucketsRespsList()) {
if (resp.hasErrorCode()) {
bucketToOffsetMap
.get(resp.getBucketId())
.completeExceptionally(
ApiError.fromErrorMessage(resp)
.exception());
} else {
bucketToOffsetMap
.get(resp.getBucketId())
.complete(resp.getOffset());
}
}
});
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,130 @@
/*
* Copyright (c) 2025 Alibaba Group Holding Ltd.
*
* Licensed 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 com.alibaba.fluss.client.admin.internal;

import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
import java.util.stream.Collectors;

/* This file is based on source code of Apache Kafka Project (https://kafka.apache.org/), licensed by the Apache
* Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
* additional information regarding copyright ownership. */

/** A future that can be used to track the completion of a set of lookup keys. */
public interface AdminApiFuture<K, V> {

/**
* The initial set of lookup keys. Although this will usually match the fulfillment keys, it
* does not necessarily have to. For example, in the case of {@link
* BucketLeaderTabletServerLookupStrategy}, we use the lookup phase in order to discover the set
* of keys that will be searched during the fulfillment phase.
*
* @return non-empty set of initial lookup keys
*/
Set<K> lookupKeys();

/**
* Complete the futures associated with the given keys.
*
* @param values the completed keys with their respective values
*/
void complete(Map<K, V> values);

/**
* Invoked when lookup of a set of keys succeeds.
*
* @param serverIdMapping the discovered mapping from key to the respective serverId that will
* handle the fulfillment request
*/
default void completeLookup(Map<K, Integer> serverIdMapping) {}

/**
* Invoked when lookup fails with a fatal error on a set of keys.
*
* @param lookupErrors the set of keys that failed lookup with their respective errors
*/
default void completeLookupExceptionally(Map<K, Throwable> lookupErrors) {
completeExceptionally(lookupErrors);
}

/**
* Complete the futures associated with the given keys exceptionally.
*
* @param errors the failed keys with their respective errors
*/
void completeExceptionally(Map<K, Throwable> errors);

static <K, V> SimpleAdminApiFuture<K, V> forKeys(Set<K> keys) {
return new SimpleAdminApiFuture<>(keys);
}

/** This class can be used when the set of keys is known ahead of time. */
class SimpleAdminApiFuture<K, V> implements AdminApiFuture<K, V> {
private final Map<K, CompletableFuture<V>> futures;

public SimpleAdminApiFuture(Set<K> keys) {
this.futures =
keys.stream()
.collect(
Collectors.toMap(
Function.identity(), k -> new CompletableFuture<>()));
}

@Override
public Set<K> lookupKeys() {
return futures.keySet();
}

@Override
public void complete(Map<K, V> values) {
values.forEach(this::complete);
}

private void complete(K key, V value) {
futureOrThrow(key).complete(value);
}

@Override
public void completeExceptionally(Map<K, Throwable> errors) {
errors.forEach(this::completeExceptionally);
}

private void completeExceptionally(K key, Throwable t) {
futureOrThrow(key).completeExceptionally(t);
}

private CompletableFuture<V> futureOrThrow(K key) {
CompletableFuture<V> future = futures.get(key);
if (future == null) {
throw new IllegalArgumentException(
"Attempt to complete future for " + key + ", which was not requested");
} else {
return future;
}
}

public Map<K, CompletableFuture<V>> all() {
return futures;
}

public CompletableFuture<V> get(K key) {
return futures.get(key);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* Copyright (c) 2024 Alibaba Group Holding Ltd.
*
* Licensed 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 com.alibaba.fluss.client.admin.internal;

import java.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;

/* This file is based on source code of Apache Kafka Project (https://kafka.apache.org/), licensed by the Apache
* Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
* additional information regarding copyright ownership. */

/**
* A handler to lookup server id for keys and then send request to these servers.
*
* @param <K>
* @param <V>
*/
public interface AdminApiHandler<K, V> {

/**
* Get the lookup strategy that is responsible for finding the server id which will handle each
* respective key.
*
* @return non-null lookup strategy
*/
TabletServerLookupStrategy<K> lookupStrategy();

/**
* Handle the given keys by sending the respective requests to the given serverId. The handler
* should parse the response, check for errors, and return a result which indicates which keys
* (if any) have either been completed or failed with an unrecoverable error. It is also
* possible that the response indicates an incorrect target serverId (e. g. in the case of a
* NotLeader error when the request is bound for a partition leader). In this case the key will
* be "unmapped" from the target serverId and lookup will be retried. Note that keys which
* received a retriable error should be left out of the result. They will be retried
* automatically.
*
* @param node the node to send the requests to
* @param keys the keys to handle
* @return result indicating key completion, failure, and unmapping
*/
CompletableFuture<ApiResult<K, V>> handle(int node, Set<K> keys);

class ApiResult<K, V> {
public final Map<K, V> completedKeys;
public final Map<K, Throwable> failedKeys;
public final Set<K> unmappedKeys;

public ApiResult(
Map<K, V> completedKeys, Map<K, Throwable> failedKeys, Set<K> unmappedKeys) {
this.completedKeys = Collections.unmodifiableMap(completedKeys);
this.failedKeys = Collections.unmodifiableMap(failedKeys);
this.unmappedKeys = Collections.unmodifiableSet(unmappedKeys);
}
}
}
Loading

0 comments on commit 439987e

Please sign in to comment.