-
Notifications
You must be signed in to change notification settings - Fork 370
[client] Introduce AdminApiDriver for workflows which lookup broker first then send request. #479
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
loserwang1024
wants to merge
1
commit into
apache:main
Choose a base branch
from
loserwang1024:AdminApiDriver
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
130 changes: 130 additions & 0 deletions
130
fluss-client/src/main/java/com/alibaba/fluss/client/admin/internal/AdminApiFuture.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} | ||
} |
72 changes: 72 additions & 0 deletions
72
fluss-client/src/main/java/com/alibaba/fluss/client/admin/internal/AdminApiHandler.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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); | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.