3333import org .apache .fluss .exception .InvalidDatabaseException ;
3434import org .apache .fluss .exception .InvalidTableException ;
3535import org .apache .fluss .exception .LakeTableAlreadyExistException ;
36+ import org .apache .fluss .exception .NonPrimaryKeyTableException ;
3637import org .apache .fluss .exception .SecurityDisabledException ;
3738import org .apache .fluss .exception .TableAlreadyExistException ;
3839import org .apache .fluss .exception .TableNotPartitionedException ;
5051import org .apache .fluss .metadata .TableBucket ;
5152import org .apache .fluss .metadata .TableChange ;
5253import org .apache .fluss .metadata .TableDescriptor ;
54+ import org .apache .fluss .metadata .TableInfo ;
5355import org .apache .fluss .metadata .TablePath ;
5456import org .apache .fluss .rpc .gateway .CoordinatorGateway ;
5557import org .apache .fluss .rpc .messages .AcquireKvSnapshotLeaseRequest ;
@@ -300,6 +302,11 @@ public void authorizeTable(OperationType operationType, long tableId) {
300302 */
301303 private void authorizeTableWithSession (
302304 Session session , OperationType operationType , long tableId ) {
305+ TablePath tablePath = getTablePathById (tableId );
306+ authorizer .authorize (session , operationType , Resource .table (tablePath ));
307+ }
308+
309+ private TablePath getTablePathById (long tableId ) {
303310 TablePath tablePath ;
304311 try {
305312 // TODO: this will block on the coordinator event thread, consider refactor
@@ -320,8 +327,16 @@ private void authorizeTableWithSession(
320327 + "metadata cache in the server is not updated yet." ,
321328 name (), tableId ));
322329 }
330+ return tablePath ;
331+ }
323332
324- authorizer .authorize (session , operationType , Resource .table (tablePath ));
333+ private void validateKvTable (long tableId ) {
334+ TablePath tablePath = getTablePathById (tableId );
335+ TableInfo tableInfo = metadataManager .getTable (tablePath );
336+ if (!tableInfo .hasPrimaryKey ()) {
337+ throw new NonPrimaryKeyTableException (
338+ "Table '" + tablePath + "' is not a primary key table" );
339+ }
325340 }
326341
327342 @ Override
@@ -881,13 +896,15 @@ public CompletableFuture<ControlledShutdownResponse> controlledShutdown(
881896 @ Override
882897 public CompletableFuture <AcquireKvSnapshotLeaseResponse > acquireKvSnapshotLease (
883898 AcquireKvSnapshotLeaseRequest request ) {
884- // Authorization: require WRITE permission on all tables in the request
885- if ( authorizer != null ) {
886- for ( PbKvSnapshotLeaseForTable kvSnapshotLeaseForTable :
887- request . getSnapshotsToLeasesList () ) {
888- long tableId = kvSnapshotLeaseForTable . getTableId ();
899+ for ( PbKvSnapshotLeaseForTable kvSnapshotLeaseForTable :
900+ request . getSnapshotsToLeasesList () ) {
901+ long tableId = kvSnapshotLeaseForTable . getTableId ();
902+ if ( authorizer != null ) {
903+ // Authorization: require WRITE permission on all tables in the request
889904 authorizeTable (OperationType .READ , tableId );
890905 }
906+
907+ validateKvTable (tableId );
891908 }
892909
893910 String leaseId = request .getLeaseId ();
@@ -913,12 +930,14 @@ public CompletableFuture<AcquireKvSnapshotLeaseResponse> acquireKvSnapshotLease(
913930 @ Override
914931 public CompletableFuture <ReleaseKvSnapshotLeaseResponse > releaseKvSnapshotLease (
915932 ReleaseKvSnapshotLeaseRequest request ) {
916- // Authorization: require WRITE permission on all tables in the request
917- if ( authorizer != null ) {
918- for ( PbTableBucket tableBucket : request . getBucketsToReleasesList () ) {
919- long tableId = tableBucket . getTableId ();
933+ for ( PbTableBucket tableBucket : request . getBucketsToReleasesList ()) {
934+ long tableId = tableBucket . getTableId ();
935+ if ( authorizer != null ) {
936+ // Authorization: require WRITE permission on all tables in the request.
920937 authorizeTable (OperationType .READ , tableId );
921938 }
939+
940+ validateKvTable (tableId );
922941 }
923942
924943 String leaseId = request .getLeaseId ();
0 commit comments