swuferhong commented on code in PR #2126: URL: https://github.com/apache/fluss/pull/2126#discussion_r2600823984
########## fluss-server/src/main/java/org/apache/fluss/server/utils/DatabaseLimitResolver.java: ########## @@ -0,0 +1,62 @@ +/* + * 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.server.utils; + +import org.apache.fluss.config.cluster.ConfigEntry; + +import java.util.List; +import java.util.Objects; + +import static org.apache.fluss.server.DynamicServerConfig.DATABASE_LIMITS_PREFIX; + +/** + * Analyze the database-level quota upper limit, and calculate the effective bucket and partition + * upper limit of each database based on cluster configuration and dynamic configuration. + */ +public final class DatabaseLimitResolver { Review Comment: Adding tests to cover this class. ########## fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java: ########## @@ -279,7 +284,12 @@ public long createTable( boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException { // validate table properties before creating table - validateTableDescriptor(tableToCreate, maxBucketNum); + int dbLevelMaxBucket = + DatabaseLimitResolver.resolveMaxBucketForDb( + maxBucketNum, + dynamicConfigManager.describeConfigs(), + tablePath.getDatabaseName()); + validateTableDescriptor(tableToCreate, maxBucketNum, dbLevelMaxBucket); Review Comment: We only need to pass the value returned by `DatabaseLimitResolver.resolveMaxBucketForDb` into `validateTableDescriptor`, so the separate `maxBucketNum` parameter can be removed. Additionally, could we rename the return value of `DatabaseLimitResolver.resolveMaxBucketForDb` to `maxBucketNum` for clarity? ########## fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java: ########## @@ -712,12 +727,26 @@ public void createPartition( if (totalBuckets > maxBucketNum) { throw new TooManyBucketsException( String.format( - "Adding partition '%s' would result in %d total buckets for table %s, exceeding the maximum of %d buckets.", + "Adding partition '%s' would result in %d total buckets for table %s, exceeding the cluster-level maximum of %d buckets.", Review Comment: This check can be removed, only retain the check after resolved by `DatabaseLimitResolver`. ########## fluss-server/src/main/java/org/apache/fluss/server/utils/TableDescriptorValidation.java: ########## @@ -163,9 +165,15 @@ private static void checkDistribution(TableDescriptor tableDescriptor, int maxBu if (bucketCount > maxBucketNum) { throw new TooManyBucketsException( String.format( - "Bucket count %s exceeds the maximum limit %s.", + "Bucket count %s exceeds the cluster-level maximum limit %s.", bucketCount, maxBucketNum)); Review Comment: This check can be removed. -- 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]
