2626import org .apache .fluss .metrics .registry .MetricRegistry ;
2727import org .apache .fluss .rpc .GatewayClientProxy ;
2828import org .apache .fluss .rpc .RpcClient ;
29+ import org .apache .fluss .rpc .gateway .AdminReadOnlyGateway ;
2930import org .apache .fluss .rpc .gateway .CoordinatorGateway ;
3031import org .apache .fluss .rpc .messages .CommitLakeTableSnapshotRequest ;
32+ import org .apache .fluss .rpc .messages .PbBucketOffset ;
3133import org .apache .fluss .rpc .messages .PbCommitLakeTableSnapshotRespForTable ;
3234import org .apache .fluss .rpc .messages .PbLakeTableOffsetForBucket ;
3335import org .apache .fluss .rpc .messages .PbLakeTableSnapshotInfo ;
3436import org .apache .fluss .rpc .messages .PbLakeTableSnapshotMetadata ;
3537import org .apache .fluss .rpc .messages .PbPrepareCommitLakeTableRespForTable ;
38+ import org .apache .fluss .rpc .messages .PbTableBucketOffsets ;
3639import org .apache .fluss .rpc .messages .PrepareCommitLakeTableSnapshotRequest ;
3740import org .apache .fluss .rpc .messages .PrepareCommitLakeTableSnapshotResponse ;
3841import org .apache .fluss .rpc .metrics .ClientMetricGroup ;
@@ -66,6 +69,7 @@ public class FlussTableLakeSnapshotCommitter implements AutoCloseable {
6669 private final Configuration flussConf ;
6770
6871 private CoordinatorGateway coordinatorGateway ;
72+ private AdminReadOnlyGateway readOnlyGateway ;
6973 private RpcClient rpcClient ;
7074
7175 public FlussTableLakeSnapshotCommitter (Configuration flussConf ) {
@@ -83,38 +87,40 @@ public void open() {
8387 this .coordinatorGateway =
8488 GatewayClientProxy .createGatewayProxy (
8589 metadataUpdater ::getCoordinatorServer , rpcClient , CoordinatorGateway .class );
90+
91+ this .readOnlyGateway =
92+ GatewayClientProxy .createGatewayProxy (
93+ metadataUpdater ::getRandomTabletServer ,
94+ rpcClient ,
95+ AdminReadOnlyGateway .class );
8696 }
8797
8898 String prepareCommit (long tableId , TablePath tablePath , Map <TableBucket , Long > logEndOffsets )
8999 throws IOException {
90- PbPrepareCommitLakeTableRespForTable prepareCommitResp = null ;
91- Exception exception = null ;
100+ PbPrepareCommitLakeTableRespForTable prepareCommitResp ;
92101 try {
93102 PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest =
94103 toPrepareCommitLakeTableSnapshotRequest (tableId , tablePath , logEndOffsets );
95104 PrepareCommitLakeTableSnapshotResponse prepareCommitLakeTableSnapshotResponse =
96- coordinatorGateway
105+ readOnlyGateway
97106 .prepareCommitLakeTableSnapshot (prepareCommitLakeTableSnapshotRequest )
98107 .get ();
99108 List <PbPrepareCommitLakeTableRespForTable > pbPrepareCommitLakeTableRespForTables =
100109 prepareCommitLakeTableSnapshotResponse .getPrepareCommitLakeTableRespsList ();
101110 checkState (pbPrepareCommitLakeTableRespForTables .size () == 1 );
102111 prepareCommitResp = pbPrepareCommitLakeTableRespForTables .get (0 );
103112 if (prepareCommitResp .hasErrorCode ()) {
104- exception = ApiError .fromErrorMessage (prepareCommitResp ).exception ();
113+ throw ApiError .fromErrorMessage (prepareCommitResp ).exception ();
114+ } else {
115+ return checkNotNull (prepareCommitResp ).getLakeTableSnapshotFilePath ();
105116 }
106117 } catch (Exception e ) {
107- exception = e ;
108- }
109-
110- if (exception != null ) {
111118 throw new IOException (
112119 String .format (
113120 "Fail to prepare commit table lake snapshot for %s to Fluss." ,
114121 tablePath ),
115- ExceptionUtils .stripExecutionException (exception ));
122+ ExceptionUtils .stripExecutionException (e ));
116123 }
117- return checkNotNull (prepareCommitResp ).getLakeTableSnapshotFilePath ();
118124 }
119125
120126 void commit (
@@ -124,7 +130,6 @@ void commit(
124130 Map <TableBucket , Long > logEndOffsets ,
125131 Map <TableBucket , Long > logMaxTieredTimestamps )
126132 throws IOException {
127- Exception exception = null ;
128133 try {
129134 CommitLakeTableSnapshotRequest request =
130135 toCommitLakeTableSnapshotRequest (
@@ -139,13 +144,9 @@ void commit(
139144 PbCommitLakeTableSnapshotRespForTable commitLakeTableSnapshotRes =
140145 commitLakeTableSnapshotRespForTables .get (0 );
141146 if (commitLakeTableSnapshotRes .hasErrorCode ()) {
142- exception = ApiError .fromErrorMessage (commitLakeTableSnapshotRes ).exception ();
147+ throw ApiError .fromErrorMessage (commitLakeTableSnapshotRes ).exception ();
143148 }
144- } catch (Exception e ) {
145- exception = e ;
146- }
147-
148- if (exception != null ) {
149+ } catch (Exception exception ) {
149150 throw new IOException (
150151 String .format (
151152 "Fail to commit table lake snapshot id %d of table %d to Fluss." ,
@@ -166,26 +167,22 @@ private PrepareCommitLakeTableSnapshotRequest toPrepareCommitLakeTableSnapshotRe
166167 long tableId , TablePath tablePath , Map <TableBucket , Long > logEndOffsets ) {
167168 PrepareCommitLakeTableSnapshotRequest prepareCommitLakeTableSnapshotRequest =
168169 new PrepareCommitLakeTableSnapshotRequest ();
169- PbLakeTableSnapshotInfo pbLakeTableSnapshotInfo =
170- prepareCommitLakeTableSnapshotRequest .addTablesReq ();
171- pbLakeTableSnapshotInfo .setTableId (tableId );
170+ PbTableBucketOffsets pbTableBucketOffsets =
171+ prepareCommitLakeTableSnapshotRequest .addBucketOffset ();
172+ pbTableBucketOffsets .setTableId (tableId );
173+ pbTableBucketOffsets
174+ .setTablePath ()
175+ .setDatabaseName (tablePath .getDatabaseName ())
176+ .setTableName (tablePath .getTableName ());
172177
173- // in prepare phase, we don't know the snapshot id,
174- // set -1 since the field is required
175- pbLakeTableSnapshotInfo .setSnapshotId (-1L );
176178 for (Map .Entry <TableBucket , Long > logEndOffsetEntry : logEndOffsets .entrySet ()) {
177- PbLakeTableOffsetForBucket pbLakeTableOffsetForBucket =
178- pbLakeTableSnapshotInfo .addBucketsReq ();
179+ PbBucketOffset pbBucketOffset = pbTableBucketOffsets .addBucketOffset ();
179180 TableBucket tableBucket = logEndOffsetEntry .getKey ();
180- pbLakeTableSnapshotInfo
181- .setTablePath ()
182- .setDatabaseName (tablePath .getDatabaseName ())
183- .setTableName (tablePath .getTableName ());
184181 if (tableBucket .getPartitionId () != null ) {
185- pbLakeTableOffsetForBucket .setPartitionId (tableBucket .getPartitionId ());
182+ pbBucketOffset .setPartitionId (tableBucket .getPartitionId ());
186183 }
187- pbLakeTableOffsetForBucket .setBucketId (tableBucket .getBucket ());
188- pbLakeTableOffsetForBucket .setLogEndOffset (logEndOffsetEntry .getValue ());
184+ pbBucketOffset .setBucketId (tableBucket .getBucket ());
185+ pbBucketOffset .setLogEndOffset (logEndOffsetEntry .getValue ());
189186 }
190187 return prepareCommitLakeTableSnapshotRequest ;
191188 }
0 commit comments