This is an automated email from the ASF dual-hosted git repository.

hongshun pushed a commit to branch release-0.9
in repository https://gitbox.apache.org/repos/asf/fluss.git


The following commit(s) were added to refs/heads/release-0.9 by this push:
     new 2d8d89936 [server] Fix ListDatabaseSummary always return table count 
as 0 or 1(#2852) (#2899)
2d8d89936 is described below

commit 2d8d89936537c8d9566f4851b5c99c97dda33b82
Author: Hongshun Wang <[email protected]>
AuthorDate: Thu Mar 19 10:11:19 2026 +0800

    [server] Fix ListDatabaseSummary always return table count as 0 or 1(#2852) 
(#2899)
---
 .../fluss/client/admin/FlussAdminITCase.java       |  2 +-
 .../apache/fluss/server/zk/ZooKeeperClient.java    | 51 +++++++++++++++-------
 .../fluss/server/zk/ZooKeeperClientTest.java       | 44 ++++++++++++++++---
 3 files changed, 74 insertions(+), 23 deletions(-)

diff --git 
a/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java
 
b/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java
index 2d1416409..946c558e1 100644
--- 
a/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java
+++ 
b/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java
@@ -900,7 +900,7 @@ class FlussAdminITCase extends ClientToServerITCaseBase {
                                 Collectors.toMap(
                                         DatabaseSummary::getDatabaseName,
                                         DatabaseSummary::getTableCount));
-        assertThat(databaseSummaries.get("db1")).isEqualTo(1);
+        assertThat(databaseSummaries.get("db1")).isEqualTo(2);
         assertThat(databaseSummaries.get("db2")).isEqualTo(0);
 
         assertThatThrownBy(() -> admin.listTables("unknown_db").get())
diff --git 
a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java 
b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java
index d088e103c..e6efb053e 100644
--- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java
+++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java
@@ -495,31 +495,52 @@ public class ZooKeeperClient implements AutoCloseable {
 
     public List<DatabaseSummary> listDatabaseSummaries(Collection<String> 
databaseNames)
             throws Exception {
-        Map<String, String> path2DatabaseNamesMap =
+        Map<String, String> dbPathToDatabaseName =
                 databaseNames.stream()
                         .collect(toMap(DatabaseZNode::path, databaseName -> 
databaseName));
-        List<ZkCheckExistsResponse> statsInBackground =
-                getStatInBackground(path2DatabaseNamesMap.keySet());
+        Map<String, String> tablesPathToDatabaseName =
+                databaseNames.stream()
+                        .collect(toMap(TablesZNode::path, databaseName -> 
databaseName));
+        List<String> requestPaths = new 
ArrayList<>(dbPathToDatabaseName.keySet());
+        requestPaths.addAll(tablesPathToDatabaseName.keySet());
+        List<ZkCheckExistsResponse> statResponses = 
getStatInBackground(requestPaths);
+
         List<DatabaseSummary> databaseSummaries = new ArrayList<>();
-        for (ZkCheckExistsResponse response : statsInBackground) {
+
+        Map<String, Long> dbCreatedTimes = new HashMap<>();
+        Map<String, Integer> dbTableCounts = new HashMap<>();
+        for (ZkCheckExistsResponse response : statResponses) {
             Stat stat = response.getStat();
+            String path = response.getPath();
             if (!response.hasError() && stat != null) {
-                // To decrease the cost, use zk node creation time as the 
database creation
-                // time rather than create_time in node data.
-                databaseSummaries.add(
-                        new DatabaseSummary(
-                                path2DatabaseNamesMap.get(response.getPath()),
-                                response.getStat().getCtime(),
-                                response.getStat().getNumChildren()));
+                if (dbPathToDatabaseName.containsKey(path)) {
+                    // Use zk node creation time as the database creation time 
to avoid reading
+                    // node data.
+                    dbCreatedTimes.put(dbPathToDatabaseName.get(path), 
stat.getCtime());
+                } else {
+                    dbTableCounts.put(tablesPathToDatabaseName.get(path), 
stat.getNumChildren());
+                }
+            } else if 
(response.getResultCode().equals(KeeperException.Code.NONODE)
+                    && tablesPathToDatabaseName.containsKey(path)) {
+                dbTableCounts.put(tablesPathToDatabaseName.get(path), 0);
             } else {
-                // silently ignore the database which does not exist anymore,
-                // because the database names are listed by server not user
                 LOG.warn(
-                        "Failed to get database summary for database {}. {}",
-                        path2DatabaseNamesMap.get(response.getPath()),
+                        "Failed to get database summary for database {}: {}",
+                        path,
                         response.getErrorMessage());
             }
         }
+
+        for (String databaseName : databaseNames) {
+            if (dbCreatedTimes.containsKey(databaseName)
+                    && dbTableCounts.containsKey(databaseName)) {
+                databaseSummaries.add(
+                        new DatabaseSummary(
+                                databaseName,
+                                dbCreatedTimes.get(databaseName),
+                                dbTableCounts.get(databaseName)));
+            }
+        }
         return databaseSummaries;
     }
 
diff --git 
a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java
 
b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java
index 4cf0a511e..fd47d9173 100644
--- 
a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java
+++ 
b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java
@@ -703,11 +703,13 @@ class ZooKeeperClientTest {
 
     @Test
     void testGetDatabaseSummary() throws Exception {
-        TablePath tablePath = TablePath.of("db", "tb1");
+        TablePath tablePath1 = TablePath.of("db", "tb1");
+        TablePath tablePath2 = TablePath.of("db", "tb2");
+        TablePath tablePath3 = TablePath.of("db2", "tb1");
 
         assertThat(
                         zookeeperClient.listDatabaseSummaries(
-                                
Collections.singletonList(tablePath.getDatabaseName())))
+                                
Collections.singletonList(tablePath1.getDatabaseName())))
                 .isEmpty();
 
         // register table.
@@ -722,18 +724,46 @@ class ZooKeeperClientTest {
                         Collections.emptyMap(),
                         beforeCreateTime,
                         beforeCreateTime);
-        zookeeperClient.registerTable(tablePath, tableReg1);
-
+        zookeeperClient.registerTable(tablePath1, tableReg1);
         long afterCreateTime = System.currentTimeMillis();
+
+        TableRegistration tableReg2 =
+                new TableRegistration(
+                        12,
+                        "second table",
+                        Arrays.asList("a", "b"),
+                        new TableDescriptor.TableDistribution(16, 
Collections.singletonList("a")),
+                        Collections.emptyMap(),
+                        Collections.emptyMap(),
+                        beforeCreateTime,
+                        beforeCreateTime);
+        zookeeperClient.registerTable(tablePath2, tableReg2);
+
+        TableRegistration tableReg3 =
+                new TableRegistration(
+                        13,
+                        "third table",
+                        Arrays.asList("a", "b"),
+                        new TableDescriptor.TableDistribution(16, 
Collections.singletonList("a")),
+                        Collections.emptyMap(),
+                        Collections.emptyMap(),
+                        beforeCreateTime,
+                        beforeCreateTime);
+        zookeeperClient.registerTable(tablePath3, tableReg3);
+
         List<DatabaseSummary> databaseSummaries =
                 zookeeperClient.listDatabaseSummaries(
-                        
Collections.singletonList(tablePath.getDatabaseName()));
-        assertThat(databaseSummaries).hasSize(1);
+                        Arrays.asList(tablePath1.getDatabaseName(), 
tablePath3.getDatabaseName()));
+        assertThat(databaseSummaries).hasSize(2);
         DatabaseSummary databaseSummary = databaseSummaries.get(0);
         assertThat(databaseSummary.getDatabaseName()).isEqualTo("db");
-        assertThat(databaseSummary.getTableCount()).isEqualTo(1);
+        assertThat(databaseSummary.getTableCount()).isEqualTo(2);
         assertThat(databaseSummary.getCreatedTime())
                 .isGreaterThanOrEqualTo(beforeCreateTime)
                 .isLessThanOrEqualTo(afterCreateTime);
+        databaseSummary = databaseSummaries.get(1);
+        assertThat(databaseSummary.getDatabaseName()).isEqualTo("db2");
+        assertThat(databaseSummary.getTableCount()).isEqualTo(1);
+        
assertThat(databaseSummary.getCreatedTime()).isGreaterThan(afterCreateTime);
     }
 }

Reply via email to