3838import com .alibaba .fluss .rpc .GatewayClientProxy ;
3939import com .alibaba .fluss .rpc .RpcClient ;
4040import com .alibaba .fluss .rpc .gateway .AdminGateway ;
41+ import com .alibaba .fluss .rpc .gateway .AdminReadOnlyGateway ;
4142import com .alibaba .fluss .rpc .gateway .TabletServerGateway ;
4243import com .alibaba .fluss .rpc .messages .CreateAclsRequest ;
4344import com .alibaba .fluss .rpc .messages .CreateDatabaseRequest ;
99100public class FlussAdmin implements Admin {
100101
101102 private final AdminGateway gateway ;
103+ private final AdminReadOnlyGateway readOnlyGateway ;
102104 private final MetadataUpdater metadataUpdater ;
103105 private final RpcClient client ;
104106
105107 public FlussAdmin (RpcClient client , MetadataUpdater metadataUpdater ) {
106108 this .gateway =
107109 GatewayClientProxy .createGatewayProxy (
108110 metadataUpdater ::getCoordinatorServer , client , AdminGateway .class );
111+ this .readOnlyGateway =
112+ GatewayClientProxy .createGatewayProxy (
113+ metadataUpdater ::getRandomTabletServer , client , AdminGateway .class );
109114 this .metadataUpdater = metadataUpdater ;
110115 this .client = client ;
111116 }
@@ -119,7 +124,7 @@ public CompletableFuture<List<ServerNode>> getServerNodes() {
119124 List <ServerNode > serverNodeList = new ArrayList <>();
120125 Cluster cluster =
121126 sendMetadataRequestAndRebuildCluster (
122- gateway ,
127+ readOnlyGateway ,
123128 false ,
124129 metadataUpdater .getCluster (),
125130 null ,
@@ -142,7 +147,8 @@ public CompletableFuture<SchemaInfo> getTableSchema(TablePath tablePath) {
142147 request .setTablePath ()
143148 .setDatabaseName (tablePath .getDatabaseName ())
144149 .setTableName (tablePath .getTableName ());
145- return gateway .getTableSchema (request )
150+ return readOnlyGateway
151+ .getTableSchema (request )
146152 .thenApply (
147153 r ->
148154 new SchemaInfo (
@@ -157,7 +163,8 @@ public CompletableFuture<SchemaInfo> getTableSchema(TablePath tablePath, int sch
157163 .setTablePath ()
158164 .setDatabaseName (tablePath .getDatabaseName ())
159165 .setTableName (tablePath .getTableName ());
160- return gateway .getTableSchema (request )
166+ return readOnlyGateway
167+ .getTableSchema (request )
161168 .thenApply (
162169 r ->
163170 new SchemaInfo (
@@ -179,7 +186,8 @@ public CompletableFuture<Void> createDatabase(
179186 public CompletableFuture <DatabaseInfo > getDatabaseInfo (String databaseName ) {
180187 GetDatabaseInfoRequest request = new GetDatabaseInfoRequest ();
181188 request .setDatabaseName (databaseName );
182- return gateway .getDatabaseInfo (request )
189+ return readOnlyGateway
190+ .getDatabaseInfo (request )
183191 .thenApply (
184192 r ->
185193 new DatabaseInfo (
@@ -204,13 +212,14 @@ public CompletableFuture<Void> dropDatabase(
204212 public CompletableFuture <Boolean > databaseExists (String databaseName ) {
205213 DatabaseExistsRequest request = new DatabaseExistsRequest ();
206214 request .setDatabaseName (databaseName );
207- return gateway .databaseExists (request ).thenApply (DatabaseExistsResponse ::isExists );
215+ return readOnlyGateway .databaseExists (request ).thenApply (DatabaseExistsResponse ::isExists );
208216 }
209217
210218 @ Override
211219 public CompletableFuture <List <String >> listDatabases () {
212220 ListDatabasesRequest request = new ListDatabasesRequest ();
213- return gateway .listDatabases (request )
221+ return readOnlyGateway
222+ .listDatabases (request )
214223 .thenApply (ListDatabasesResponse ::getDatabaseNamesList );
215224 }
216225
@@ -233,7 +242,8 @@ public CompletableFuture<TableInfo> getTableInfo(TablePath tablePath) {
233242 request .setTablePath ()
234243 .setDatabaseName (tablePath .getDatabaseName ())
235244 .setTableName (tablePath .getTableName ());
236- return gateway .getTableInfo (request )
245+ return readOnlyGateway
246+ .getTableInfo (request )
237247 .thenApply (
238248 r ->
239249 TableInfo .of (
@@ -261,14 +271,14 @@ public CompletableFuture<Boolean> tableExists(TablePath tablePath) {
261271 request .setTablePath ()
262272 .setDatabaseName (tablePath .getDatabaseName ())
263273 .setTableName (tablePath .getTableName ());
264- return gateway .tableExists (request ).thenApply (TableExistsResponse ::isExists );
274+ return readOnlyGateway .tableExists (request ).thenApply (TableExistsResponse ::isExists );
265275 }
266276
267277 @ Override
268278 public CompletableFuture <List <String >> listTables (String databaseName ) {
269279 ListTablesRequest request = new ListTablesRequest ();
270280 request .setDatabaseName (databaseName );
271- return gateway .listTables (request ).thenApply (ListTablesResponse ::getTableNamesList );
281+ return readOnlyGateway .listTables (request ).thenApply (ListTablesResponse ::getTableNamesList );
272282 }
273283
274284 @ Override
@@ -289,7 +299,8 @@ public CompletableFuture<List<PartitionInfo>> listPartitionInfos(
289299 PbPartitionSpec pbPartitionSpec = makePbPartitionSpec (partitionSpec );
290300 request .setPartialPartitionSpec (pbPartitionSpec );
291301 }
292- return gateway .listPartitionInfos (request )
302+ return readOnlyGateway
303+ .listPartitionInfos (request )
293304 .thenApply (ClientRpcMessageUtils ::toPartitionInfos );
294305 }
295306
@@ -315,7 +326,8 @@ public CompletableFuture<KvSnapshots> getLatestKvSnapshots(TablePath tablePath)
315326 request .setTablePath ()
316327 .setDatabaseName (tablePath .getDatabaseName ())
317328 .setTableName (tablePath .getTableName ());
318- return gateway .getLatestKvSnapshots (request )
329+ return readOnlyGateway
330+ .getLatestKvSnapshots (request )
319331 .thenApply (ClientRpcMessageUtils ::toKvSnapshots );
320332 }
321333
@@ -328,7 +340,8 @@ public CompletableFuture<KvSnapshots> getLatestKvSnapshots(
328340 .setDatabaseName (tablePath .getDatabaseName ())
329341 .setTableName (tablePath .getTableName ());
330342 request .setPartitionName (partitionName );
331- return gateway .getLatestKvSnapshots (request )
343+ return readOnlyGateway
344+ .getLatestKvSnapshots (request )
332345 .thenApply (ClientRpcMessageUtils ::toKvSnapshots );
333346 }
334347
@@ -342,7 +355,8 @@ public CompletableFuture<KvSnapshotMetadata> getKvSnapshotMetadata(
342355 request .setTableId (bucket .getTableId ())
343356 .setBucketId (bucket .getBucket ())
344357 .setSnapshotId (snapshotId );
345- return gateway .getKvSnapshotMetadata (request )
358+ return readOnlyGateway
359+ .getKvSnapshotMetadata (request )
346360 .thenApply (ClientRpcMessageUtils ::toKvSnapshotMetadata );
347361 }
348362
@@ -353,7 +367,8 @@ public CompletableFuture<LakeSnapshot> getLatestLakeSnapshot(TablePath tablePath
353367 .setDatabaseName (tablePath .getDatabaseName ())
354368 .setTableName (tablePath .getTableName ());
355369
356- return gateway .getLatestLakeSnapshot (request )
370+ return readOnlyGateway
371+ .getLatestLakeSnapshot (request )
357372 .thenApply (ClientRpcMessageUtils ::toLakeTableSnapshotInfo );
358373 }
359374
0 commit comments