wuchong commented on code in PR #2515:
URL: https://github.com/apache/fluss/pull/2515#discussion_r2751185239
##########
fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java:
##########
@@ -1275,13 +1290,8 @@ public List<String> getChildren(String path) throws
Exception {
}
/** Gets the data and stat of a given zk node path. */
- public Optional<Stat> getStat(String path) throws Exception {
- try {
- Stat stat = zkClient.checkExists().forPath(path);
- return Optional.ofNullable(stat);
- } catch (KeeperException.NoNodeException e) {
- return Optional.empty();
- }
+ public Stat getStat(String path) throws Exception {
+ return zkClient.checkExists().forPath(path);
Review Comment:
Why make this change? The previous implementation was clearer and safer. The
current one doesn’t properly handle the case where the znode does not exist,
which can lead to runtime errors or undefined behavior.
##########
fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java:
##########
@@ -179,6 +180,9 @@ CompletableFuture<Void> dropDatabase(
/** List all databases in fluss cluster asynchronously. */
CompletableFuture<List<String>> listDatabases();
+ /** List all databases' summary information in fluss cluster
asynchronously. */
+ CompletableFuture<List<DatabaseSummary>> listDatabases(ListDatabaseOption
option);
Review Comment:
Rename the method to `listDatabaseSummaries()` without any parameters. If
user would like only returning database names, they can use `listDatabases()`.
##########
fluss-rpc/src/main/proto/FlussApi.proto:
##########
@@ -93,9 +93,12 @@ message DatabaseExistsResponse {
// list databases request and response
message ListDatabasesRequest {
+ optional bool database_name_only = 1;
Review Comment:
For legacy clients, this new field will be missing, which typically causes
the boolean to default to false. This creates a risk of backward
incompatibility: the server might return the full summary instead of just the
names, effectively changing the behavior for old clients.
While we could technically work around this by checking
`request.hasDatabaseNameOnly() && !request.isDatabaseNameOnly()`, this logic is
counter-intuitive and error-prone.
**Suggestion:** A better design would be to use `bool include_summary`
instead of `database_name_only`. So, for legacy clients, the default value
(`false`) and empty value (unset this field) are consistent and correctly imply
'do not include summary,' automatically preserving the existing behavior
without requiring complex null-checks.
##########
fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java:
##########
@@ -487,6 +488,20 @@ public List<String> listDatabases() throws Exception {
return getChildren(DatabasesZNode.path());
}
+ public Optional<DatabaseSummary> getDatabaseSummary(String databaseName)
throws Exception {
Review Comment:
The current implementation still triggers **N separate I/O operations**—one
for each database—which can significantly increase latency. To mitigate this,
we can optimize by performing these requests **asynchronously in the
background**.
Specifically, you can:
1. Introduce a new method in `ZooKeeperClient`:
```java
getStatsInBackground(Collection<String> paths)
```
This method should wrap the existing `handleRequestInBackground` and use
a newly defined `ZkGetStatRequest` type to fetch stats for multiple paths
concurrently. Take `getDataInBackground` as an example.
2. Then, add a new high-level method:
```java
List<DatabaseSummary> listDatabaseSummaries(Collection<String>
databaseNames)
```
which calls `getStatsInBackground` to retrieve all necessary ZooKeeper
stats in parallel and constructs the `DatabaseSummary` objects efficiently.
This approach reduces end-to-end latency by eliminating sequential I/O and
leveraging concurrent background requests.
##########
fluss-common/src/main/java/org/apache/fluss/metadata/DatabaseSummary.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.fluss.metadata;
+
+import org.apache.fluss.annotation.PublicEvolving;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+
+/**
+ * Aggregated summary information of a database for listing purposes.
+ *
+ * <p>This class contains aggregated metadata about a database, including
creation time, table
+ * count, and other summary statistics. It is distinct from {@link
DatabaseInfo} which contains
+ * complete database metadata including the {@link DatabaseDescriptor}.
+ *
+ * @since 0.6
+ */
+@PublicEvolving
+public class DatabaseSummary {
+ private final String databaseName;
+ private final @Nullable Long createdTime;
+ private final @Nullable Integer tableCount;
Review Comment:
I suggest making these fields **non-nullable** and using primitive types
(`long` and `int`) instead of their boxed counterparts.
If users need database names, they should call `listDatabases()`
directly—rather than fetching database summaries and then discarding them.
This approach makes the API clearer, more predictable, and deterministic in
behavior.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]